You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by xi...@apache.org on 2021/03/15 09:16:24 UTC

[shardingsphere] branch master updated: #9590, remove the read write split logic in shardingsphere-ha (#9671)

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

xiaoyu 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 1351a75  #9590, remove the read write split logic in shardingsphere-ha (#9671)
1351a75 is described below

commit 1351a75b774a93a1015e1d7dab786a1350fa29fb
Author: Zhang Yonglun <zh...@apache.org>
AuthorDate: Mon Mar 15 17:15:46 2021 +0800

    #9590, remove the read write split logic in shardingsphere-ha (#9671)
    
    * #9590, remove the read write split logic in shardingsphere-ha
    
    * #9590, remove the read write split logic in shardingsphere-ha
---
 .../ha/api/config/HARuleConfiguration.java         |  2 -
 .../config/rule/HADataSourceRuleConfiguration.java |  4 --
 .../ha/spi/ReplicaLoadBalanceAlgorithm.java        | 38 ---------------
 .../RandomReplicaLoadBalanceAlgorithm.java         | 46 ------------------
 .../RoundRobinReplicaLoadBalanceAlgorithm.java     | 45 -----------------
 .../AlgorithmProvidedHARuleConfiguration.java      |  3 --
 .../apache/shardingsphere/ha/constant/HAOrder.java |  4 +-
 .../shardingsphere/ha/rule/HADataSourceRule.java   | 15 ++----
 .../org/apache/shardingsphere/ha/rule/HARule.java  | 14 +-----
 .../ha/yaml/config/YamlHARuleConfiguration.java    |  2 -
 .../rule/YamlHADataSourceRuleConfiguration.java    |  4 --
 ...eAlgorithmProviderConfigurationYamlSwapper.java | 11 +----
 .../swapper/HARuleConfigurationYamlSwapper.java    | 14 +-----
 ...ardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm | 19 --------
 .../RandomReplicaLoadBalanceAlgorithmTest.java     | 41 ----------------
 .../RoundRobinReplicaLoadBalanceAlgorithmTest.java | 42 ----------------
 .../ha/rule/HADataSourceRuleTest.java              | 32 ++++++-------
 .../apache/shardingsphere/ha/rule/HARuleTest.java  | 28 +++++------
 .../AlgorithmProvidedHARuleBuilderTest.java        |  2 +-
 .../ha/rule/biulder/HARuleBuilderTest.java         |  2 +-
 ...orithmProviderConfigurationYamlSwapperTest.java | 15 +-----
 .../HARuleConfigurationYamlSwapperTest.java        | 25 ++++------
 ...nfigurationsForYamlHARuleConfigurationTest.java |  2 -
 .../src/test/resources/yaml/ha-rule.yaml           |  9 ----
 .../apache/shardingsphere/ha/mgr/MGRHAType.java    |  4 +-
 .../ha/route/engine/HASQLRouter.java               |  4 +-
 .../ha/route/engine/impl/HADataSourceRouter.java   | 21 +-------
 .../route/engine/impl/PrimaryVisitedManager.java   | 55 ---------------------
 .../ha/route/engine/HASQLRouterTest.java           | 56 ++--------------------
 .../spring/boot/HARuleSpringbootConfiguration.java | 32 ++-----------
 .../algorithm/HAAlgorithmProvidedBeanRegistry.java | 45 -----------------
 .../ha/spring/boot/HASpringBootStarterTest.java    | 13 -----
 .../src/test/resources/application-ha.properties   |  7 +--
 .../ReplicaLoadBalanceAlgorithmFactoryBean.java    | 38 ---------------
 .../namespace/handler/HANamespaceHandler.java      |  4 --
 .../parser/HARuleBeanDefinitionParser.java         | 16 -------
 .../namespace/tag/HARuleBeanDefinitionTag.java     |  8 ----
 .../tag/LoadBalanceAlgorithmBeanDefinitionTag.java | 30 ------------
 .../src/main/resources/META-INF/namespace/ha.xsd   | 14 ------
 .../src/test/resources/yaml/ha-rule.yaml           |  2 -
 .../yaml/registryCenter/data-ha-rule.yaml          |  2 -
 41 files changed, 68 insertions(+), 702 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/HARuleConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/HARuleConfiguration.java
index 0417b4a..c8c0c21 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/HARuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/HARuleConfiguration.java
@@ -35,7 +35,5 @@ public final class HARuleConfiguration implements RuleConfiguration {
     
     private final Collection<HADataSourceRuleConfiguration> dataSources;
     
-    private final Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers;
-    
     private final Map<String, ShardingSphereAlgorithmConfiguration> haTypes;
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/rule/HADataSourceRuleConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/rule/HADataSourceRuleConfiguration.java
index 027b3b5..06d4330 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/rule/HADataSourceRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/rule/HADataSourceRuleConfiguration.java
@@ -33,9 +33,5 @@ public final class HADataSourceRuleConfiguration {
     
     private final List<String> dataSourceNames;
     
-    private final String loadBalancerName;
-    
-    private final boolean replicaQuery;
-    
     private final String haTypeName;
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/spi/ReplicaLoadBalanceAlgorithm.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/spi/ReplicaLoadBalanceAlgorithm.java
deleted file mode 100644
index 7a16e27..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/spi/ReplicaLoadBalanceAlgorithm.java
+++ /dev/null
@@ -1,38 +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.ha.spi;
-
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithm;
-
-import java.util.List;
-
-/**
- * Replica load-balance algorithm.
- */
-public interface ReplicaLoadBalanceAlgorithm extends ShardingSphereAlgorithm {
-    
-    /**
-     * Get data source.
-     * 
-     * @param name HA logic data source name
-     * @param primaryDataSourceName name of primary data sources
-     * @param dataSourceNames names of replica data sources
-     * @return name of selected data source
-     */
-    String getDataSource(String name, String primaryDataSourceName, List<String> dataSourceNames);
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithm.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithm.java
deleted file mode 100644
index f651cec..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithm.java
+++ /dev/null
@@ -1,46 +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.ha.algorithm;
-
-import lombok.Getter;
-import lombok.Setter;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
-
-import java.util.List;
-import java.util.Properties;
-import java.util.concurrent.ThreadLocalRandom;
-
-/**
- * Random replica load-balance algorithm.
- */
-@Getter
-@Setter
-public final class RandomReplicaLoadBalanceAlgorithm implements ReplicaLoadBalanceAlgorithm {
-    
-    private Properties props = new Properties();
-    
-    @Override
-    public String getDataSource(final String name, final String primaryDataSourceName, final List<String> dataSourceNames) {
-        return dataSourceNames.get(ThreadLocalRandom.current().nextInt(dataSourceNames.size()));
-    }
-    
-    @Override
-    public String getType() {
-        return "RANDOM";
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithm.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithm.java
deleted file mode 100644
index 213721f..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithm.java
+++ /dev/null
@@ -1,45 +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.ha.algorithm;
-
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
-
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Round-robin replica load-balance algorithm.
- */
-public final class RoundRobinReplicaLoadBalanceAlgorithm implements ReplicaLoadBalanceAlgorithm {
-    
-    private static final ConcurrentHashMap<String, AtomicInteger> COUNTS = new ConcurrentHashMap<>();
-    
-    @Override
-    public String getDataSource(final String name, final String primaryDataSourceName, final List<String> dataSourceNames) {
-        AtomicInteger count = COUNTS.containsKey(name) ? COUNTS.get(name) : new AtomicInteger(0);
-        COUNTS.putIfAbsent(name, count);
-        count.compareAndSet(dataSourceNames.size(), 0);
-        return dataSourceNames.get(Math.abs(count.getAndIncrement()) % dataSourceNames.size());
-    }
-    
-    @Override
-    public String getType() {
-        return "ROUND_ROBIN";
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/config/AlgorithmProvidedHARuleConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/config/AlgorithmProvidedHARuleConfiguration.java
index 36e399d..8cfd180 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/config/AlgorithmProvidedHARuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/config/AlgorithmProvidedHARuleConfiguration.java
@@ -23,7 +23,6 @@ import lombok.NoArgsConstructor;
 import lombok.Setter;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.apache.shardingsphere.ha.spi.HAType;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 
 import java.util.Collection;
@@ -42,7 +41,5 @@ public final class AlgorithmProvidedHARuleConfiguration implements RuleConfigura
     
     private Collection<HADataSourceRuleConfiguration> dataSources = new LinkedList<>();
     
-    private Map<String, ReplicaLoadBalanceAlgorithm> loadBalanceAlgorithms = new LinkedHashMap<>();
-    
     private Map<String, HAType> haTypes = new LinkedHashMap<>();
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/constant/HAOrder.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/constant/HAOrder.java
index a25036d..d15d355 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/constant/HAOrder.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/constant/HAOrder.java
@@ -29,10 +29,10 @@ public final class HAOrder {
     /**
      * HA order.
      */
-    public static final int ORDER = 20;
+    public static final int ORDER = 2;
     
     /**
      * Algorithm provider HA order.
      */
-    public static final int ALGORITHM_PROVIDER_HA_ORDER = 21;
+    public static final int ALGORITHM_PROVIDER_HA_ORDER = 3;
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HADataSourceRule.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HADataSourceRule.java
index 511ded0..3f13514 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HADataSourceRule.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HADataSourceRule.java
@@ -23,7 +23,6 @@ import lombok.Getter;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.apache.shardingsphere.ha.spi.HAType;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
 
 import java.util.Collection;
 import java.util.HashMap;
@@ -44,34 +43,28 @@ public final class HADataSourceRule {
     
     private final List<String> dataSourceNames;
     
-    private final ReplicaLoadBalanceAlgorithm loadBalancer;
-    
-    private final boolean replicaQuery;
-    
     private final HAType haType;
     
     private final Collection<String> disabledDataSourceNames = new HashSet<>();
     
     private String primaryDataSourceName;
     
-    public HADataSourceRule(final HADataSourceRuleConfiguration config, final ReplicaLoadBalanceAlgorithm loadBalancer, final HAType haType) {
+    public HADataSourceRule(final HADataSourceRuleConfiguration config, final HAType haType) {
         checkConfiguration(config);
         name = config.getName();
         dataSourceNames = config.getDataSourceNames();
-        this.loadBalancer = loadBalancer;
-        replicaQuery = config.isReplicaQuery();
         this.haType = haType;
     }
     
     private void checkConfiguration(final HADataSourceRuleConfiguration config) {
         Preconditions.checkArgument(!Strings.isNullOrEmpty(config.getName()), "Name is required.");
-        Preconditions.checkArgument(null != config.getDataSourceNames() && !config.getDataSourceNames().isEmpty(), "Replica data source names are required.");
+        Preconditions.checkArgument(null != config.getDataSourceNames() && !config.getDataSourceNames().isEmpty(), "Data source names are required.");
     }
     
     /**
-     * Get replica data source names.
+     * Get data source names.
      *
-     * @return available replica data source names
+     * @return available data source names
      */
     public List<String> getDataSourceNames() {
         return dataSourceNames.stream().filter(each -> !disabledDataSourceNames.contains(each)).collect(Collectors.toList());
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HARule.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HARule.java
index 83c0604..8ed2ac8 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HARule.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/HARule.java
@@ -33,7 +33,6 @@ import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
@@ -50,12 +49,9 @@ import java.util.Optional;
 public final class HARule implements DataSourceContainedRule, StatusContainedRule {
     
     static {
-        ShardingSphereServiceLoader.register(ReplicaLoadBalanceAlgorithm.class);
         ShardingSphereServiceLoader.register(HAType.class);
     }
     
-    private final Map<String, ReplicaLoadBalanceAlgorithm> loadBalancers = new LinkedHashMap<>();
-    
     private final Map<String, HAType> haTypes = new LinkedHashMap<>();
     
     private final Map<String, HADataSourceRule> dataSourceRules;
@@ -64,15 +60,12 @@ public final class HARule implements DataSourceContainedRule, StatusContainedRul
         Preconditions.checkArgument(!config.getDataSources().isEmpty(), "HA data source rules can not be empty.");
         Preconditions.checkArgument(null != dataSourceMap && !dataSourceMap.isEmpty(), "Data sources cannot be empty.");
         Preconditions.checkArgument(null != databaseType, "Database type cannot be null.");
-        config.getLoadBalancers().forEach((key, value) -> loadBalancers.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, ReplicaLoadBalanceAlgorithm.class)));
         config.getHaTypes().forEach((key, value) -> haTypes.put(key, ShardingSphereAlgorithmFactory.createAlgorithm(value, HAType.class)));
         dataSourceRules = new HashMap<>(config.getDataSources().size(), 1);
         for (HADataSourceRuleConfiguration each : config.getDataSources()) {
-            ReplicaLoadBalanceAlgorithm loadBalanceAlgorithm = Strings.isNullOrEmpty(each.getLoadBalancerName()) || !loadBalancers.containsKey(each.getLoadBalancerName())
-                    ? TypedSPIRegistry.getRegisteredService(ReplicaLoadBalanceAlgorithm.class) : loadBalancers.get(each.getLoadBalancerName());
             HAType haType = Strings.isNullOrEmpty(each.getHaTypeName()) || !haTypes.containsKey(each.getHaTypeName())
                     ? TypedSPIRegistry.getRegisteredService(HAType.class) : haTypes.get(each.getHaTypeName());
-            dataSourceRules.put(each.getName(), new HADataSourceRule(each, loadBalanceAlgorithm, haType));
+            dataSourceRules.put(each.getName(), new HADataSourceRule(each, haType));
         }
         for (Entry<String, HADataSourceRule> entry : dataSourceRules.entrySet()) {
             String groupName = entry.getKey();
@@ -97,14 +90,11 @@ public final class HARule implements DataSourceContainedRule, StatusContainedRul
         Preconditions.checkArgument(!config.getDataSources().isEmpty(), "HA data source rules can not be empty.");
         Preconditions.checkArgument(null != dataSourceMap && !dataSourceMap.isEmpty(), "Data sources cannot be empty.");
         Preconditions.checkArgument(null != databaseType, "Database type cannot be null.");
-        loadBalancers.putAll(config.getLoadBalanceAlgorithms());
         dataSourceRules = new HashMap<>(config.getDataSources().size(), 1);
         for (HADataSourceRuleConfiguration each : config.getDataSources()) {
-            ReplicaLoadBalanceAlgorithm loadBalanceAlgorithm = Strings.isNullOrEmpty(each.getLoadBalancerName()) || !loadBalancers.containsKey(each.getLoadBalancerName())
-                    ? TypedSPIRegistry.getRegisteredService(ReplicaLoadBalanceAlgorithm.class) : loadBalancers.get(each.getLoadBalancerName());
             HAType haType = Strings.isNullOrEmpty(each.getHaTypeName()) || !haTypes.containsKey(each.getHaTypeName())
                     ? TypedSPIRegistry.getRegisteredService(HAType.class) : haTypes.get(each.getHaTypeName());
-            dataSourceRules.put(each.getName(), new HADataSourceRule(each, loadBalanceAlgorithm, haType));
+            dataSourceRules.put(each.getName(), new HADataSourceRule(each, haType));
         }
         for (Entry<String, HADataSourceRule> entry : dataSourceRules.entrySet()) {
             String groupName = entry.getKey();
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/YamlHARuleConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/YamlHARuleConfiguration.java
index d109626..9bcf14c 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/YamlHARuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/YamlHARuleConfiguration.java
@@ -36,8 +36,6 @@ public final class YamlHARuleConfiguration implements YamlRuleConfiguration {
     
     private Map<String, YamlHADataSourceRuleConfiguration> dataSources = new LinkedHashMap<>();
     
-    private Map<String, YamlShardingSphereAlgorithmConfiguration> loadBalancers = new LinkedHashMap<>();
-    
     private Map<String, YamlShardingSphereAlgorithmConfiguration> haTypes = new LinkedHashMap<>();
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/rule/YamlHADataSourceRuleConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/rule/YamlHADataSourceRuleConfiguration.java
index e11c4aa..217b40e 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/rule/YamlHADataSourceRuleConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/config/rule/YamlHADataSourceRuleConfiguration.java
@@ -36,10 +36,6 @@ public final class YamlHADataSourceRuleConfiguration implements YamlConfiguratio
     
     private List<String> dataSourceNames = new ArrayList<>();
     
-    private String loadBalancerName;
-    
-    private boolean replicaQuery;
-    
     private String haTypeName;
     
     private Properties props = new Properties();
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapper.java
index 47095ec..082df76 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapper.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapper.java
@@ -17,13 +17,12 @@
 
 package org.apache.shardingsphere.ha.yaml.swapper;
 
-import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
-import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapper;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.apache.shardingsphere.ha.constant.HAOrder;
 import org.apache.shardingsphere.ha.yaml.config.YamlHARuleConfiguration;
 import org.apache.shardingsphere.ha.yaml.config.rule.YamlHADataSourceRuleConfiguration;
+import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapper;
 
 import java.util.Collection;
 import java.util.LinkedHashMap;
@@ -42,9 +41,6 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapper
         YamlHARuleConfiguration result = new YamlHARuleConfiguration();
         result.setDataSources(data.getDataSources().stream().collect(
                 Collectors.toMap(HADataSourceRuleConfiguration::getName, this::swapToYamlConfiguration, (oldValue, currentValue) -> oldValue, LinkedHashMap::new)));
-        if (null != data.getLoadBalanceAlgorithms()) {
-            data.getLoadBalanceAlgorithms().forEach((key, value) -> result.getLoadBalancers().put(key, new YamlShardingSphereAlgorithmConfiguration(value.getType(), value.getProps())));
-        }
         return result;
     }
     
@@ -52,8 +48,6 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapper
         YamlHADataSourceRuleConfiguration result = new YamlHADataSourceRuleConfiguration();
         result.setName(dataSourceRuleConfig.getName());
         result.setDataSourceNames(dataSourceRuleConfig.getDataSourceNames());
-        result.setLoadBalancerName(dataSourceRuleConfig.getLoadBalancerName());
-        result.setReplicaQuery(dataSourceRuleConfig.isReplicaQuery());
         return result;
     }
     
@@ -69,8 +63,7 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapper
     }
     
     private HADataSourceRuleConfiguration swapToObject(final String name, final YamlHADataSourceRuleConfiguration yamlDataSourceRuleConfig) {
-        return new HADataSourceRuleConfiguration(name, yamlDataSourceRuleConfig.getDataSourceNames(),
-                yamlDataSourceRuleConfig.getLoadBalancerName(), yamlDataSourceRuleConfig.isReplicaQuery(), yamlDataSourceRuleConfig.getHaTypeName());
+        return new HADataSourceRuleConfiguration(name, yamlDataSourceRuleConfig.getDataSourceNames(), yamlDataSourceRuleConfig.getHaTypeName());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapper.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapper.java
index 45a0775..02ab8aa 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapper.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapper.java
@@ -46,9 +46,6 @@ public final class HARuleConfigurationYamlSwapper
         YamlHARuleConfiguration result = new YamlHARuleConfiguration();
         result.setDataSources(data.getDataSources().stream().collect(
                 Collectors.toMap(HADataSourceRuleConfiguration::getName, this::swapToYamlConfiguration, (oldValue, currentValue) -> oldValue, LinkedHashMap::new)));
-        if (null != data.getLoadBalancers()) {
-            data.getLoadBalancers().forEach((key, value) -> result.getLoadBalancers().put(key, algorithmSwapper.swapToYamlConfiguration(value)));
-        }
         if (null != data.getHaTypes()) {
             data.getHaTypes().forEach((key, value) -> result.getHaTypes().put(key, algorithmSwapper.swapToYamlConfiguration(value)));
         }
@@ -59,8 +56,6 @@ public final class HARuleConfigurationYamlSwapper
         YamlHADataSourceRuleConfiguration result = new YamlHADataSourceRuleConfiguration();
         result.setName(dataSourceRuleConfig.getName());
         result.setDataSourceNames(dataSourceRuleConfig.getDataSourceNames());
-        result.setLoadBalancerName(dataSourceRuleConfig.getLoadBalancerName());
-        result.setReplicaQuery(dataSourceRuleConfig.isReplicaQuery());
         result.setHaTypeName(dataSourceRuleConfig.getHaTypeName());
         return result;
     }
@@ -71,20 +66,15 @@ public final class HARuleConfigurationYamlSwapper
         for (Entry<String, YamlHADataSourceRuleConfiguration> entry : yamlConfig.getDataSources().entrySet()) {
             dataSources.add(swapToObject(entry.getKey(), entry.getValue()));
         }
-        Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers = new LinkedHashMap<>(yamlConfig.getLoadBalancers().entrySet().size(), 1);
-        if (null != yamlConfig.getLoadBalancers()) {
-            yamlConfig.getLoadBalancers().forEach((key, value) -> loadBalancers.put(key, algorithmSwapper.swapToObject(value)));
-        }
         Map<String, ShardingSphereAlgorithmConfiguration> haTypes = new LinkedHashMap<>(yamlConfig.getHaTypes().entrySet().size(), 1);
         if (null != yamlConfig.getHaTypes()) {
             yamlConfig.getHaTypes().forEach((key, value) -> haTypes.put(key, algorithmSwapper.swapToObject(value)));
         }
-        return new HARuleConfiguration(dataSources, loadBalancers, haTypes);
+        return new HARuleConfiguration(dataSources, haTypes);
     }
     
     private HADataSourceRuleConfiguration swapToObject(final String name, final YamlHADataSourceRuleConfiguration yamlDataSourceRuleConfig) {
-        return new HADataSourceRuleConfiguration(name, yamlDataSourceRuleConfig.getDataSourceNames(),
-                yamlDataSourceRuleConfig.getLoadBalancerName(), yamlDataSourceRuleConfig.isReplicaQuery(), yamlDataSourceRuleConfig.getHaTypeName());
+        return new HADataSourceRuleConfiguration(name, yamlDataSourceRuleConfig.getDataSourceNames(), yamlDataSourceRuleConfig.getHaTypeName());
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm
deleted file mode 100644
index 1394608..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm
+++ /dev/null
@@ -1,19 +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.
-#
-
-org.apache.shardingsphere.ha.algorithm.RoundRobinReplicaLoadBalanceAlgorithm
-org.apache.shardingsphere.ha.algorithm.RandomReplicaLoadBalanceAlgorithm
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithmTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithmTest.java
deleted file mode 100644
index 68d21b0..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RandomReplicaLoadBalanceAlgorithmTest.java
+++ /dev/null
@@ -1,41 +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.ha.algorithm;
-
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-import static org.junit.Assert.assertTrue;
-
-public final class RandomReplicaLoadBalanceAlgorithmTest {
-    
-    private final RandomReplicaLoadBalanceAlgorithm randomReplicaLoadBalanceAlgorithm = new RandomReplicaLoadBalanceAlgorithm();
-    
-    @Test
-    public void assertGetDataSource() {
-        String primaryDataSourceName = "test_primary_ds";
-        String replicaDataSourceName1 = "test_replica_ds_1";
-        String replicaDataSourceName2 = "test_replica_ds_2";
-        List<String> dataSourceNames = Arrays.asList(replicaDataSourceName1, replicaDataSourceName2);
-        assertTrue(dataSourceNames.contains(randomReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames)));
-        assertTrue(dataSourceNames.contains(randomReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames)));
-        assertTrue(dataSourceNames.contains(randomReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames)));
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithmTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithmTest.java
deleted file mode 100644
index 2e8cfcd..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/algorithm/RoundRobinReplicaLoadBalanceAlgorithmTest.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.ha.algorithm;
-
-import org.junit.Test;
-
-import java.util.Arrays;
-import java.util.List;
-
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-
-public final class RoundRobinReplicaLoadBalanceAlgorithmTest {
-    
-    private final RoundRobinReplicaLoadBalanceAlgorithm roundRobinReplicaLoadBalanceAlgorithm = new RoundRobinReplicaLoadBalanceAlgorithm();
-    
-    @Test
-    public void assertGetDataSource() {
-        String primaryDataSourceName = "test_primary_ds";
-        String replicaDataSourceName1 = "test_replica_ds_1";
-        String replicaDataSourceName2 = "test_replica_ds_2";
-        List<String> dataSourceNames = Arrays.asList(replicaDataSourceName1, replicaDataSourceName2);
-        assertThat(roundRobinReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames), is(replicaDataSourceName1));
-        assertThat(roundRobinReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames), is(replicaDataSourceName2));
-        assertThat(roundRobinReplicaLoadBalanceAlgorithm.getDataSource("ds", primaryDataSourceName, dataSourceNames), is(replicaDataSourceName1));
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HADataSourceRuleTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HADataSourceRuleTest.java
index dec87fe..57dbb16 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HADataSourceRuleTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HADataSourceRuleTest.java
@@ -19,8 +19,6 @@ package org.apache.shardingsphere.ha.rule;
 
 import com.google.common.collect.ImmutableMap;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
-import org.apache.shardingsphere.ha.algorithm.RandomReplicaLoadBalanceAlgorithm;
-import org.apache.shardingsphere.ha.algorithm.RoundRobinReplicaLoadBalanceAlgorithm;
 import org.apache.shardingsphere.ha.mgr.MGRHAType;
 import org.junit.Test;
 
@@ -35,51 +33,51 @@ import static org.junit.Assert.assertThat;
 public final class HADataSourceRuleTest {
     
     private final HADataSourceRule haDataSourceRule = new HADataSourceRule(
-            new HADataSourceRuleConfiguration("test_pr", Arrays.asList("replica_ds_0", "replica_ds_1"), "random", true, "haTypeName"), new RandomReplicaLoadBalanceAlgorithm(), new MGRHAType());
+            new HADataSourceRuleConfiguration("test_pr", Arrays.asList("ds_0", "ds_1"), "haTypeName"), new MGRHAType());
     
     @Test(expected = IllegalArgumentException.class)
     public void assertNewHADataSourceRuleWithoutName() {
-        new HADataSourceRule(new HADataSourceRuleConfiguration("", Collections.singletonList("replica_ds"), null, true, "haTypeName"), new RoundRobinReplicaLoadBalanceAlgorithm(), new MGRHAType());
+        new HADataSourceRule(new HADataSourceRuleConfiguration("", Arrays.asList("ds_0", "ds_1"), "haTypeName"), new MGRHAType());
     }
     
     @Test(expected = IllegalArgumentException.class)
-    public void assertNewHADataSourceRuleWithNullReplicaDataSourceName() {
-        new HADataSourceRule(new HADataSourceRuleConfiguration("ds", null, null, true, "haTypeName"), new RoundRobinReplicaLoadBalanceAlgorithm(), new MGRHAType());
+    public void assertNewHADataSourceRuleWithNullDataSourceName() {
+        new HADataSourceRule(new HADataSourceRuleConfiguration("ds", null, "haTypeName"), new MGRHAType());
     }
     
     @Test(expected = IllegalArgumentException.class)
-    public void assertNewHADataSourceRuleWithEmptyReplicaDataSourceName() {
-        new HADataSourceRule(new HADataSourceRuleConfiguration("ds", Collections.emptyList(), null, true, "haTypeName"), new RoundRobinReplicaLoadBalanceAlgorithm(), new MGRHAType());
+    public void assertNewHADataSourceRuleWithEmptyDataSourceName() {
+        new HADataSourceRule(new HADataSourceRuleConfiguration("ds", Collections.emptyList(), "haTypeName"), new MGRHAType());
     }
     
     @Test
     public void assertGetDataSourceNamesWithoutDisabledDataSourceNames() {
-        assertThat(haDataSourceRule.getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
+        assertThat(haDataSourceRule.getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
     public void assertGetDataSourceNamesWithDisabledDataSourceNames() {
-        haDataSourceRule.updateDisabledDataSourceNames("replica_ds_0", true);
-        assertThat(haDataSourceRule.getDataSourceNames(), is(Collections.singletonList("replica_ds_1")));
+        haDataSourceRule.updateDisabledDataSourceNames("ds_0", true);
+        assertThat(haDataSourceRule.getDataSourceNames(), is(Collections.singletonList("ds_1")));
     }
     
     @Test
     public void assertUpdateDisabledDataSourceNamesForDisabled() {
-        haDataSourceRule.updateDisabledDataSourceNames("replica_ds_0", true);
-        assertThat(haDataSourceRule.getDataSourceNames(), is(Collections.singletonList("replica_ds_1")));
+        haDataSourceRule.updateDisabledDataSourceNames("ds_0", true);
+        assertThat(haDataSourceRule.getDataSourceNames(), is(Collections.singletonList("ds_1")));
     }
     
     @Test
     public void assertUpdateDisabledDataSourceNamesForEnabled() {
-        haDataSourceRule.updateDisabledDataSourceNames("replica_ds_0", true);
-        haDataSourceRule.updateDisabledDataSourceNames("replica_ds_0", false);
-        assertThat(haDataSourceRule.getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
+        haDataSourceRule.updateDisabledDataSourceNames("ds_0", true);
+        haDataSourceRule.updateDisabledDataSourceNames("ds_0", false);
+        assertThat(haDataSourceRule.getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
     public void assertGetDataSourceMapper() {
         Map<String, Collection<String>> actual = haDataSourceRule.getDataSourceMapper();
-        Map<String, Collection<String>> expected = ImmutableMap.of("test_pr", Arrays.asList("replica_ds_0", "replica_ds_1"));
+        Map<String, Collection<String>> expected = ImmutableMap.of("test_pr", Arrays.asList("ds_0", "ds_1"));
         assertThat(actual, is(expected));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HARuleTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HARuleTest.java
index 9840e4f..83a81dc 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HARuleTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/HARuleTest.java
@@ -44,7 +44,7 @@ public final class HARuleTest {
     
     @Test(expected = IllegalArgumentException.class)
     public void assertNewWithEmptyDataSourceRule() {
-        new HARule(new HARuleConfiguration(Collections.emptyList(), Collections.emptyMap(), Collections.emptyMap()), mock(DatabaseType.class), dataSourceMap, "ha_db");
+        new HARule(new HARuleConfiguration(Collections.emptyList(), Collections.emptyMap()), mock(DatabaseType.class), dataSourceMap, "ha_db");
     }
     
     @Test
@@ -61,47 +61,45 @@ public final class HARuleTest {
     
     private HARule createHARule() {
         HADataSourceRuleConfiguration config =
-                new HADataSourceRuleConfiguration("test_pr", Arrays.asList("replica_ds_0", "replica_ds_1"), "random", true, "haTypeName");
+                new HADataSourceRuleConfiguration("test_pr", Arrays.asList("ds_0", "ds_1"), "haTypeName");
         return new HARule(new HARuleConfiguration(
-                Collections.singleton(config), ImmutableMap.of("random", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties())),
-                ImmutableMap.of("mgr", new ShardingSphereAlgorithmConfiguration("MGR", new Properties()))),
+                Collections.singleton(config), ImmutableMap.of("mgr", new ShardingSphereAlgorithmConfiguration("MGR", new Properties()))),
                 mock(DatabaseType.class), dataSourceMap, "ha_db");
     }
     
     private void assertDataSourceRule(final HADataSourceRule actual) {
         assertThat(actual.getName(), is("test_pr"));
-        assertThat(actual.getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
-        assertThat(actual.getLoadBalancer().getType(), is("RANDOM"));
+        assertThat(actual.getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
     public void assertUpdateRuleStatusWithNotExistDataSource() {
         HARule haRule = createHARule();
-        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("replica_db", true));
-        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
+        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("db", true));
+        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
     public void assertUpdateRuleStatus() {
         HARule haRule = createHARule();
-        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("replica_ds_0", true));
-        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Collections.singletonList("replica_ds_1")));
+        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("ds_0", true));
+        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Collections.singletonList("ds_1")));
     }
     
     @Test
     public void assertUpdateRuleStatusWithEnable() {
         HARule haRule = createHARule();
-        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("replica_ds_0", true));
-        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Collections.singletonList("replica_ds_1")));
-        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("replica_ds_0", false));
-        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
+        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("ds_0", true));
+        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Collections.singletonList("ds_1")));
+        haRule.updateRuleStatus(new DataSourceNameDisabledEvent("ds_0", false));
+        assertThat(haRule.getSingleDataSourceRule().getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
     public void assertGetDataSourceMapper() {
         HARule haRule = createHARule();
         Map<String, Collection<String>> actual = haRule.getDataSourceMapper();
-        Map<String, Collection<String>> expected = ImmutableMap.of("test_pr", Arrays.asList("replica_ds_0", "replica_ds_1"));
+        Map<String, Collection<String>> expected = ImmutableMap.of("test_pr", Arrays.asList("ds_0", "ds_1"));
         assertThat(actual, is(expected));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilderTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilderTest.java
index 5e2eee2..e364663 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilderTest.java
@@ -43,7 +43,7 @@ public final class AlgorithmProvidedHARuleBuilderTest {
     public void assertBuild() {
         AlgorithmProvidedHARuleConfiguration algorithmProvidedRuleConfig = mock(AlgorithmProvidedHARuleConfiguration.class);
         HADataSourceRuleConfiguration ruleConfig = new HADataSourceRuleConfiguration(
-                "name", Collections.singletonList("name"), "loadBalancerName", true, "haTypeName");
+                "name", Collections.singletonList("name"), "haTypeName");
         when(algorithmProvidedRuleConfig.getDataSources()).thenReturn(Collections.singletonList(ruleConfig));
         ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(
                 Collections.singletonList(algorithmProvidedRuleConfig), ShardingSphereRuleBuilder.class).get(algorithmProvidedRuleConfig);
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilderTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilderTest.java
index 574faa0..87e278c 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilderTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilderTest.java
@@ -46,7 +46,7 @@ public final class HARuleBuilderTest {
     public void assertBuild() {
         HARuleConfiguration ruleConfig = mock(HARuleConfiguration.class);
         HADataSourceRuleConfiguration dataSourceRuleConfig = new HADataSourceRuleConfiguration(
-                "name", Collections.singletonList("name"), "loadBalancerName", true, "haTypeName");
+                "name", Collections.singletonList("name"), "haTypeName");
         when(ruleConfig.getDataSources()).thenReturn(Collections.singletonList(dataSourceRuleConfig));
         ShardingSphereRuleBuilder builder = OrderedSPIRegistry.getRegisteredServices(Collections.singletonList(ruleConfig), ShardingSphereRuleBuilder.class).get(ruleConfig);
         Map<String, DataSource> dataSourceMap = new HashMap<>();
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapperTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapperTest.java
index 4a78f8c..6274951 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapperTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleAlgorithmProviderConfigurationYamlSwapperTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.ha.yaml.swapper;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.shardingsphere.ha.algorithm.RandomReplicaLoadBalanceAlgorithm;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.apache.shardingsphere.ha.constant.HAOrder;
@@ -45,12 +44,6 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapperTest {
         assertNotNull(actual.getDataSources());
         assertThat(actual.getDataSources().keySet(), is(Collections.singleton("name")));
         assertThat(actual.getDataSources().get("name").getName(), is("name"));
-        assertThat(actual.getDataSources().get("name").getLoadBalancerName(), is("loadBalancerName"));
-        assertThat(actual.getDataSources().get("name").getDataSourceNames(), is(Collections.singletonList("replicaDataSourceName")));
-        assertNotNull(actual.getLoadBalancers());
-        assertThat(actual.getLoadBalancers().keySet(), is(Collections.singleton("name")));
-        assertNotNull(actual.getLoadBalancers().get("name"));
-        assertThat(actual.getLoadBalancers().get("name").getType(), is("RANDOM"));
     }
     
     @Test
@@ -62,9 +55,6 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapperTest {
         HADataSourceRuleConfiguration ruleConfig = actual.getDataSources().iterator().next();
         assertNotNull(ruleConfig);
         assertThat(ruleConfig.getName(), is("name"));
-        assertThat(ruleConfig.getLoadBalancerName(), is("loadBalancerName"));
-        assertThat(ruleConfig.getDataSourceNames(), is(Collections.singletonList("replicaDataSourceName")));
-        assertThat(actual.getLoadBalanceAlgorithms(), is(Collections.emptyMap()));
     }
     
     @Test
@@ -84,9 +74,8 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapperTest {
     
     private YamlHARuleConfiguration createYamlHARuleConfiguration() {
         HADataSourceRuleConfiguration ruleConfig = new HADataSourceRuleConfiguration("name",
-                Collections.singletonList("replicaDataSourceName"), "loadBalancerName", true, "haTypeName");
+                Collections.singletonList("dataSourceNames"), "haTypeName");
         return swapper.swapToYamlConfiguration(
-                new AlgorithmProvidedHARuleConfiguration(Collections.singletonList(ruleConfig), ImmutableMap.of("name", new RandomReplicaLoadBalanceAlgorithm()),
-                        ImmutableMap.of("mgr", new MGRHAType())));
+                new AlgorithmProvidedHARuleConfiguration(Collections.singletonList(ruleConfig), ImmutableMap.of("mgr", new MGRHAType())));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapperTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapperTest.java
index 760a8e7..67cb8df 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapperTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/HARuleConfigurationYamlSwapperTest.java
@@ -35,7 +35,6 @@ import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
@@ -50,32 +49,27 @@ public final class HARuleConfigurationYamlSwapperTest {
     @Test
     public void assertSwapToYamlWithLoadBalanceAlgorithm() {
         HADataSourceRuleConfiguration dataSourceConfig =
-                new HADataSourceRuleConfiguration("ds", Collections.singletonList("replica"), "roundRobin", true, "haTypeName");
-        YamlHARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToYamlConfiguration(new HARuleConfiguration(
-                Collections.singleton(dataSourceConfig), ImmutableMap.of("roundRobin", new ShardingSphereAlgorithmConfiguration("ROUND_ROBIN", new Properties())),
-                ImmutableMap.of("roundRobin", new ShardingSphereAlgorithmConfiguration("MGR", new Properties()))));
+                new HADataSourceRuleConfiguration("ds", Collections.singletonList("dataSourceName"), "haTypeName");
+        YamlHARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToYamlConfiguration(new HARuleConfiguration(Collections.singleton(dataSourceConfig),
+                ImmutableMap.of("mgr", new ShardingSphereAlgorithmConfiguration("MGR", new Properties()))));
         assertThat(actual.getDataSources().get("ds").getName(), is("ds"));
-        assertThat(actual.getDataSources().get("ds").getDataSourceNames(), is(Collections.singletonList("replica")));
-        assertThat(actual.getDataSources().get("ds").getLoadBalancerName(), is("roundRobin"));
+        assertThat(actual.getDataSources().get("ds").getDataSourceNames(), is(Collections.singletonList("dataSourceName")));
     }
     
     @Test
     public void assertSwapToYamlWithoutLoadBalanceAlgorithm() {
-        HADataSourceRuleConfiguration dataSourceConfig = new HADataSourceRuleConfiguration("ds", Collections.singletonList("replica"), null, true, "haTypeName");
+        HADataSourceRuleConfiguration dataSourceConfig = new HADataSourceRuleConfiguration("ds", Collections.singletonList("dataSourceName"), "haTypeName");
         YamlHARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToYamlConfiguration(
-                new HARuleConfiguration(Collections.singleton(dataSourceConfig), Collections.emptyMap(), Collections.emptyMap()));
+                new HARuleConfiguration(Collections.singleton(dataSourceConfig), Collections.emptyMap()));
         assertThat(actual.getDataSources().get("ds").getName(), is("ds"));
-        assertThat(actual.getDataSources().get("ds").getDataSourceNames(), is(Collections.singletonList("replica")));
-        assertNull(actual.getDataSources().get("ds").getLoadBalancerName());
+        assertThat(actual.getDataSources().get("ds").getDataSourceNames(), is(Collections.singletonList("dataSourceName")));
     }
     
     @Test
     public void assertSwapToObjectWithLoadBalanceAlgorithmType() {
         YamlHARuleConfiguration yamlConfig = createYamlHARuleConfiguration();
-        yamlConfig.getDataSources().get("ha_ds").setLoadBalancerName("RANDOM");
         HARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToObject(yamlConfig);
         assertHARuleConfiguration(actual);
-        assertThat(actual.getDataSources().iterator().next().getLoadBalancerName(), is("RANDOM"));
     }
     
     @Test
@@ -83,21 +77,20 @@ public final class HARuleConfigurationYamlSwapperTest {
         YamlHARuleConfiguration yamlConfig = createYamlHARuleConfiguration();
         HARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToObject(yamlConfig);
         assertHARuleConfiguration(actual);
-        assertNull(actual.getDataSources().iterator().next().getLoadBalancerName());
     }
     
     private YamlHARuleConfiguration createYamlHARuleConfiguration() {
         YamlHARuleConfiguration result = new YamlHARuleConfiguration();
         result.getDataSources().put("ha_ds", new YamlHADataSourceRuleConfiguration());
         result.getDataSources().get("ha_ds").setName("ha_ds");
-        result.getDataSources().get("ha_ds").setDataSourceNames(Arrays.asList("replica_ds_0", "replica_ds_1"));
+        result.getDataSources().get("ha_ds").setDataSourceNames(Arrays.asList("ds_0", "ds_1"));
         return result;
     }
     
     private void assertHARuleConfiguration(final HARuleConfiguration actual) {
         HADataSourceRuleConfiguration group = actual.getDataSources().iterator().next();
         assertThat(group.getName(), is("ha_ds"));
-        assertThat(group.getDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
+        assertThat(group.getDataSourceNames(), is(Arrays.asList("ds_0", "ds_1")));
     }
     
     @Test
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/YamlRootRuleConfigurationsForYamlHARuleConfigurationTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/YamlRootRuleConfigurationsForYamlHARuleConfigurationTest.java
index 8c1cdca..9be5754 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/YamlRootRuleConfigurationsForYamlHARuleConfigurationTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/yaml/swapper/YamlRootRuleConfigurationsForYamlHARuleConfigurationTest.java
@@ -70,11 +70,9 @@ public final class YamlRootRuleConfigurationsForYamlHARuleConfigurationTest {
     
     private void assertHARuleForDs0(final YamlHARuleConfiguration actual) {
         assertThat(actual.getDataSources().get("ds_0").getDataSourceNames(), is(Arrays.asList("primary_ds_0_replica_0", "primary_ds_0_replica_1")));
-        assertThat(actual.getDataSources().get("ds_0").getLoadBalancerName(), is("roundRobin"));
     }
     
     private void assertHARuleForDs1(final YamlHARuleConfiguration actual) {
         assertThat(actual.getDataSources().get("ds_1").getDataSourceNames(), is(Arrays.asList("primary_ds_1_replica_0", "primary_ds_1_replica_1")));
-        assertThat(actual.getDataSources().get("ds_1").getLoadBalancerName(), is("random"));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/yaml/ha-rule.yaml b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/yaml/ha-rule.yaml
index 9b5240d..e45f03b 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/yaml/ha-rule.yaml
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/yaml/ha-rule.yaml
@@ -60,21 +60,12 @@ rules:
       dataSourceNames:
         - primary_ds_0_replica_0
         - primary_ds_0_replica_1
-      loadBalancerName: roundRobin
-      replicaQuery: true
       haTypeName: mgr
     ds_1:
       dataSourceNames:
         - primary_ds_1_replica_0
         - primary_ds_1_replica_1
-      loadBalancerName: random
-      replicaQuery: false
       haTypeName: mgr
-  loadBalancers:
-    roundRobin:
-      type: ROUND_ROBIN
-    random:
-      type: RANDOM
   haTypes:
     mgr:
       type: MGR
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-provider/shardingsphere-ha-mgr/src/main/java/org/apache/shardingsphere/ha/mgr/MGRHAType.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-provider/shardingsphere-ha-mgr/src/main/java/org/apache/shardingsphere/ha/mgr/MGRHAType.java
index 420fa9f..3bd82cc 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-provider/shardingsphere-ha-mgr/src/main/java/org/apache/shardingsphere/ha/mgr/MGRHAType.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-provider/shardingsphere-ha-mgr/src/main/java/org/apache/shardingsphere/ha/mgr/MGRHAType.java
@@ -75,7 +75,7 @@ public final class MGRHAType implements HAType {
         try (Connection connection = dataSourceMap.get(oldPrimaryDataSource).getConnection();
              Statement statement = connection.createStatement()) {
             checkPluginIsActive(statement);
-            checkReplicaMemberCount(statement);
+            checkMemberCount(statement);
             checkServerGroupName(statement);
             checkIsSinglePrimaryMode(statement);
         }
@@ -91,7 +91,7 @@ public final class MGRHAType implements HAType {
         }
     }
     
-    private void checkReplicaMemberCount(final Statement statement) throws SQLException {
+    private void checkMemberCount(final Statement statement) throws SQLException {
         try (ResultSet resultSet = statement.executeQuery(MEMBER_COUNT)) {
             while (resultSet.next()) {
                 if (resultSet.getInt(1) < 1) {
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/HASQLRouter.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/HASQLRouter.java
index ce73d30..be21016 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/HASQLRouter.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/HASQLRouter.java
@@ -43,7 +43,7 @@ public final class HASQLRouter implements SQLRouter<HARule> {
     @Override
     public RouteContext createRouteContext(final LogicSQL logicSQL, final ShardingSphereMetaData metaData, final HARule rule, final ConfigurationProperties props) {
         RouteContext result = new RouteContext();
-        String dataSourceName = new HADataSourceRouter(rule.getSingleDataSourceRule()).route(logicSQL.getSqlStatementContext().getSqlStatement());
+        String dataSourceName = new HADataSourceRouter(rule.getSingleDataSourceRule()).route();
         result.getRouteUnits().add(new RouteUnit(new RouteMapper(DefaultSchema.LOGIC_NAME, dataSourceName), Collections.emptyList()));
         return result;
     }
@@ -58,7 +58,7 @@ public final class HASQLRouter implements SQLRouter<HARule> {
             Optional<HADataSourceRule> dataSourceRule = rule.findDataSourceRule(dataSourceName);
             if (dataSourceRule.isPresent() && dataSourceRule.get().getName().equalsIgnoreCase(each.getDataSourceMapper().getActualName())) {
                 toBeRemoved.add(each);
-                String actualDataSourceName = new HADataSourceRouter(dataSourceRule.get()).route(logicSQL.getSqlStatementContext().getSqlStatement());
+                String actualDataSourceName = new HADataSourceRouter(dataSourceRule.get()).route();
                 toBeAdded.add(new RouteUnit(new RouteMapper(each.getDataSourceMapper().getLogicName(), actualDataSourceName), each.getTableMappers()));
             }
         }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/HADataSourceRouter.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/HADataSourceRouter.java
index c1f10b6..a71c41b 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/HADataSourceRouter.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/HADataSourceRouter.java
@@ -18,11 +18,7 @@
 package org.apache.shardingsphere.ha.route.engine.impl;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.hint.HintManager;
 import org.apache.shardingsphere.ha.rule.HADataSourceRule;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.SelectStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.handler.dml.SelectStatementHandler;
 
 /**
  * Data source router for HA.
@@ -35,22 +31,9 @@ public final class HADataSourceRouter {
     /**
      * Route.
      * 
-     * @param sqlStatement SQL statement
      * @return data source name
      */
-    public String route(final SQLStatement sqlStatement) {
-        if (isPrimaryRoute(sqlStatement) || !rule.isReplicaQuery()) {
-            PrimaryVisitedManager.setPrimaryVisited();
-            return rule.getPrimaryDataSourceName();
-        }
-        return rule.getLoadBalancer().getDataSource(rule.getName(), rule.getPrimaryDataSourceName(), rule.getDataSourceNames());
-    }
-    
-    private boolean isPrimaryRoute(final SQLStatement sqlStatement) {
-        return containsLockSegment(sqlStatement) || !(sqlStatement instanceof SelectStatement) || PrimaryVisitedManager.getPrimaryVisited() || HintManager.isWriteRouteOnly();
-    }
-    
-    private boolean containsLockSegment(final SQLStatement sqlStatement) {
-        return sqlStatement instanceof SelectStatement && SelectStatementHandler.getLockSegment((SelectStatement) sqlStatement).isPresent();
+    public String route() {
+        return rule.getPrimaryDataSourceName();
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/PrimaryVisitedManager.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/PrimaryVisitedManager.java
deleted file mode 100644
index 2f2f8df..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/main/java/org/apache/shardingsphere/ha/route/engine/impl/PrimaryVisitedManager.java
+++ /dev/null
@@ -1,55 +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.ha.route.engine.impl;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-/**
- * Primary data source visited manager.
- * 
- * <p>Trace primary data source visited or not in current thread.</p>
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class PrimaryVisitedManager {
-    
-    private static final ThreadLocal<Boolean> PRIMARY_VISITED = ThreadLocal.withInitial(() -> false);
-    
-    /**
-     * Judge primary data source visited in current thread.
-     * 
-     * @return primary data source visited or not in current thread
-     */
-    public static boolean getPrimaryVisited() {
-        return PRIMARY_VISITED.get();
-    }
-    
-    /**
-     * Set primary data source visited in current thread.
-     */
-    public static void setPrimaryVisited() {
-        PRIMARY_VISITED.set(true);
-    }
-    
-    /**
-     * Clear primary data source visited.
-     */
-    public static void clear() {
-        PRIMARY_VISITED.remove();
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/engine/HASQLRouterTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/engine/HASQLRouterTest.java
index ffe5552..e80bfb0 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/engine/HASQLRouterTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/engine/HASQLRouterTest.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.ha.route.engine;
 
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
-import org.apache.shardingsphere.ha.route.engine.impl.PrimaryVisitedManager;
 import org.apache.shardingsphere.ha.rule.HARule;
 import org.apache.shardingsphere.infra.binder.LogicSQL;
 import org.apache.shardingsphere.infra.binder.statement.SQLStatementContext;
@@ -35,11 +34,7 @@ import org.apache.shardingsphere.infra.route.context.RouteMapper;
 import org.apache.shardingsphere.infra.route.context.RouteUnit;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.ordered.OrderedSPIRegistry;
-import org.apache.shardingsphere.sql.parser.sql.common.segment.dml.predicate.LockSegment;
 import org.apache.shardingsphere.sql.parser.sql.common.statement.SQLStatement;
-import org.apache.shardingsphere.sql.parser.sql.common.statement.dml.InsertStatement;
-import org.apache.shardingsphere.sql.parser.sql.dialect.statement.mysql.dml.MySQLSelectStatement;
-import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.runner.RunWith;
@@ -49,14 +44,12 @@ import org.mockito.junit.MockitoJUnitRunner;
 import javax.sql.DataSource;
 import java.util.Collections;
 import java.util.Iterator;
-import java.util.Optional;
 import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
 public final class HASQLRouterTest {
@@ -67,8 +60,6 @@ public final class HASQLRouterTest {
     
     private static final String PRIMARY_DATASOURCE = "primary";
     
-    private static final String REPLICA_DATASOURCE = "query";
-    
     private HARule rule;
     
     @Mock
@@ -82,18 +73,12 @@ public final class HASQLRouterTest {
     
     @Before
     public void setUp() {
-        rule = new HARule(new HARuleConfiguration(Collections.singleton(
-                new HADataSourceRuleConfiguration(DATASOURCE_NAME, Collections.singletonList(REPLICA_DATASOURCE), null, true, "haTypeName")),
-                Collections.emptyMap(), Collections.emptyMap()), mock(DatabaseType.class),
-                Collections.singletonMap("ds", mock(DataSource.class)), "ha_db");
+        HADataSourceRuleConfiguration haDataSourceRuleConfiguration = new HADataSourceRuleConfiguration(DATASOURCE_NAME, Collections.singletonList(PRIMARY_DATASOURCE), "haTypeName");
+        HARuleConfiguration haRuleConfiguration = new HARuleConfiguration(Collections.singleton(haDataSourceRuleConfiguration), Collections.emptyMap());
+        rule = new HARule(haRuleConfiguration, mock(DatabaseType.class), Collections.singletonMap("ds", mock(DataSource.class)), "ha_db");
         sqlRouter = (HASQLRouter) OrderedSPIRegistry.getRegisteredServices(Collections.singleton(rule), SQLRouter.class).get(rule);
     }
     
-    @After
-    public void tearDown() {
-        PrimaryVisitedManager.clear();
-    }
-    
     @Test
     public void assertCreateRouteContextToPrimaryWithoutRouteUnits() {
         LogicSQL logicSQL = new LogicSQL(mock(SQLStatementContext.class), "", Collections.emptyList());
@@ -117,38 +102,7 @@ public final class HASQLRouterTest {
     }
     
     @Test
-    public void assertCreateRouteContextToReplicaDataSource() {
-        MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
-        when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
-        when(selectStatement.getLock()).thenReturn(Optional.empty());
-        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereMetaData metaData = new ShardingSphereMetaData("logic_schema",
-                mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), mock(ShardingSphereSchema.class));
-        RouteContext actual = sqlRouter.createRouteContext(logicSQL, metaData, rule, new ConfigurationProperties(new Properties()));
-        Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
-        assertThat(routedDataSourceNames.next(), is(REPLICA_DATASOURCE));
-    }
-    
-    @Test
-    public void assertDecorateRouteContextToReplicaDataSource() {
-        RouteContext actual = mockRouteContext();
-        MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
-        when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
-        when(selectStatement.getLock()).thenReturn(Optional.empty());
-        LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
-        ShardingSphereMetaData metaData = new ShardingSphereMetaData("logic_schema",
-                mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), mock(ShardingSphereSchema.class));
-        sqlRouter.decorateRouteContext(actual, logicSQL, metaData, rule, new ConfigurationProperties(new Properties()));
-        Iterator<String> routedDataSourceNames = actual.getActualDataSourceNames().iterator();
-        assertThat(routedDataSourceNames.next(), is(NONE_HA_DATASOURCE_NAME));
-        assertThat(routedDataSourceNames.next(), is(REPLICA_DATASOURCE));
-    }
-    
-    @Test
     public void assertCreateRouteContextToPrimaryDataSourceWithLock() {
-        MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
-        when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
-        when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("logic_schema",
                 mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), mock(ShardingSphereSchema.class));
@@ -160,9 +114,6 @@ public final class HASQLRouterTest {
     @Test
     public void assertDecorateRouteContextToPrimaryDataSourceWithLock() {
         RouteContext actual = mockRouteContext();
-        MySQLSelectStatement selectStatement = mock(MySQLSelectStatement.class);
-        when(sqlStatementContext.getSqlStatement()).thenReturn(selectStatement);
-        when(selectStatement.getLock()).thenReturn(Optional.of(mock(LockSegment.class)));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("logic_schema",
                 mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), mock(ShardingSphereSchema.class));
@@ -174,7 +125,6 @@ public final class HASQLRouterTest {
     
     @Test
     public void assertCreateRouteContextToPrimaryDataSource() {
-        when(sqlStatementContext.getSqlStatement()).thenReturn(mock(InsertStatement.class));
         LogicSQL logicSQL = new LogicSQL(sqlStatementContext, "", Collections.emptyList());
         ShardingSphereMetaData metaData = new ShardingSphereMetaData("logic_schema",
                 mock(ShardingSphereResource.class, RETURNS_DEEP_STUBS), new ShardingSphereRuleMetaData(Collections.emptyList(), Collections.singleton(rule)), mock(ShardingSphereSchema.class));
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/HARuleSpringbootConfiguration.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/HARuleSpringbootConfiguration.java
index ca12a9d..54e2e80 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/HARuleSpringbootConfiguration.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/HARuleSpringbootConfiguration.java
@@ -18,25 +18,16 @@
 package org.apache.shardingsphere.ha.spring.boot;
 
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
-import org.apache.shardingsphere.ha.spring.boot.algorithm.HAAlgorithmProvidedBeanRegistry;
-import org.apache.shardingsphere.ha.spring.boot.rule.YamlHARuleSpringBootConfiguration;
 import org.apache.shardingsphere.ha.spring.boot.condition.HASpringBootCondition;
+import org.apache.shardingsphere.ha.spring.boot.rule.YamlHARuleSpringBootConfiguration;
 import org.apache.shardingsphere.ha.yaml.config.YamlHARuleConfiguration;
 import org.apache.shardingsphere.ha.yaml.swapper.HARuleAlgorithmProviderConfigurationYamlSwapper;
-import org.springframework.beans.factory.ObjectProvider;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.springframework.boot.autoconfigure.condition.ConditionalOnClass;
 import org.springframework.boot.context.properties.EnableConfigurationProperties;
 import org.springframework.context.annotation.Bean;
 import org.springframework.context.annotation.Conditional;
 import org.springframework.context.annotation.Configuration;
-import org.springframework.core.env.Environment;
-
-import java.util.Collections;
-import java.util.Map;
-import java.util.Optional;
 
 /**
  * Rule spring boot configuration for HA.
@@ -55,25 +46,10 @@ public class HARuleSpringbootConfiguration {
     /**
      * HA rule configuration for spring boot.
      *
-     * @param loadBalanceAlgorithms load balance algorithms
      * @return HA rule configuration
      */
     @Bean
-    public RuleConfiguration haRuleConfiguration(final ObjectProvider<Map<String, ReplicaLoadBalanceAlgorithm>> loadBalanceAlgorithms) {
-        AlgorithmProvidedHARuleConfiguration result = swapper.swapToObject(yamlConfig.getHa());
-        Map<String, ReplicaLoadBalanceAlgorithm> balanceAlgorithmMap = Optional.ofNullable(loadBalanceAlgorithms.getIfAvailable()).orElse(Collections.emptyMap());
-        result.setLoadBalanceAlgorithms(balanceAlgorithmMap);
-        return result;
-    }
-    
-    /**
-     * HA algorithm provided bean registry.
-     *
-     * @param environment environment
-     * @return HA algorithm provided bean registry
-     */
-    @Bean
-    public static HAAlgorithmProvidedBeanRegistry haAlgorithmProvidedBeanRegistry(final Environment environment) {
-        return new HAAlgorithmProvidedBeanRegistry(environment);
+    public RuleConfiguration haRuleConfiguration() {
+        return swapper.swapToObject(yamlConfig.getHa());
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/algorithm/HAAlgorithmProvidedBeanRegistry.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/algorithm/HAAlgorithmProvidedBeanRegistry.java
deleted file mode 100644
index 499297f..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/main/java/org/apache/shardingsphere/ha/spring/boot/algorithm/HAAlgorithmProvidedBeanRegistry.java
+++ /dev/null
@@ -1,45 +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.ha.spring.boot.algorithm;
-
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
-import org.apache.shardingsphere.spring.boot.registry.AbstractAlgorithmProvidedBeanRegistry;
-import org.springframework.beans.factory.support.BeanDefinitionRegistry;
-import org.springframework.core.env.Environment;
-
-/**
- * HA algorithm provided bean registry.
- */
-public final class HAAlgorithmProvidedBeanRegistry extends AbstractAlgorithmProvidedBeanRegistry<ReplicaLoadBalanceAlgorithm> {
-    
-    private static final String ALGORITHMS = "spring.shardingsphere.rules.ha.load-balancers.";
-    
-    /**
-     * Instantiates a new HA algorithm provided bean registry.
-     *
-     * @param environment environment
-     */
-    public HAAlgorithmProvidedBeanRegistry(final Environment environment) {
-        super(environment);
-    }
-    
-    @Override
-    public void postProcessBeanDefinitionRegistry(final BeanDefinitionRegistry registry) {
-        registerBean(ALGORITHMS, ReplicaLoadBalanceAlgorithm.class, registry);
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/java/org/apache/shardingsphere/ha/spring/boot/HASpringBootStarterTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/java/org/apache/shardingsphere/ha/spring/boot/HASpringBootStarterTest.java
index b6d18c1..b6a4179 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/java/org/apache/shardingsphere/ha/spring/boot/HASpringBootStarterTest.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/java/org/apache/shardingsphere/ha/spring/boot/HASpringBootStarterTest.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.ha.spring.boot;
 
-import org.apache.shardingsphere.ha.algorithm.RandomReplicaLoadBalanceAlgorithm;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.junit.Test;
@@ -40,27 +39,15 @@ import static org.junit.Assert.assertTrue;
 public class HASpringBootStarterTest {
     
     @Resource
-    private RandomReplicaLoadBalanceAlgorithm random;
-    
-    @Resource
     private AlgorithmProvidedHARuleConfiguration config;
     
     @Test
-    public void assertLoadBalanceAlgorithm() {
-        assertTrue(random.getProps().isEmpty());
-    }
-    
-    @Test
     public void assertHARuleConfiguration() {
         assertThat(config.getDataSources().size(), is(1));
         assertTrue(config.getDataSources().stream().findFirst().isPresent());
         HADataSourceRuleConfiguration dataSourceRuleConfig = config.getDataSources().stream().findFirst().get();
         assertThat(dataSourceRuleConfig.getName(), is("pr_ds"));
-        assertThat(dataSourceRuleConfig.getLoadBalancerName(), is("random"));
-        assertTrue(dataSourceRuleConfig.isReplicaQuery());
         assertThat(dataSourceRuleConfig.getDataSourceNames().size(), is(2));
         assertTrue(config.getDataSources().contains(dataSourceRuleConfig));
-        assertThat(config.getLoadBalanceAlgorithms().size(), is(1));
-        assertTrue(config.getLoadBalanceAlgorithms().containsKey("random"));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/resources/application-ha.properties b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/resources/application-ha.properties
index 5c7e2aa..09b1e4f 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/resources/application-ha.properties
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-boot-starter/src/test/resources/application-ha.properties
@@ -15,9 +15,6 @@
 # limitations under the License.
 #
 
-spring.shardingsphere.rules.ha.load-balancers.random.type=RANDOM
-
 spring.shardingsphere.rules.ha.data-sources.pr_ds.primary-data-source-name=primary_ds
-spring.shardingsphere.rules.ha.data-sources.pr_ds.data-source-names=replica_ds_0,replica_ds_1
-spring.shardingsphere.rules.ha.data-sources.pr_ds.load-balancer-name=random
-spring.shardingsphere.rules.ha.data-sources.pr_ds.replica-query=true
+spring.shardingsphere.rules.ha.data-sources.pr_ds.data-source-names=ds_0,ds_1
+
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/factorybean/ReplicaLoadBalanceAlgorithmFactoryBean.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/factorybean/ReplicaLoadBalanceAlgorithmFactoryBean.java
deleted file mode 100644
index b81d050..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/factorybean/ReplicaLoadBalanceAlgorithmFactoryBean.java
+++ /dev/null
@@ -1,38 +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.ha.spring.namespace.factorybean;
-
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
-import org.apache.shardingsphere.spring.namespace.factorybean.ShardingSphereAlgorithmFactoryBean;
-
-import java.util.Properties;
-
-/**
- * Replica load balance algorithm factory bean.
- */
-public final class ReplicaLoadBalanceAlgorithmFactoryBean extends ShardingSphereAlgorithmFactoryBean<ReplicaLoadBalanceAlgorithm> {
-    
-    static {
-        ShardingSphereServiceLoader.register(ReplicaLoadBalanceAlgorithm.class);
-    }
-    
-    public ReplicaLoadBalanceAlgorithmFactoryBean(final String type, final Properties props) {
-        super(ReplicaLoadBalanceAlgorithm.class, type, props);
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/handler/HANamespaceHandler.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/handler/HANamespaceHandler.java
index a2be16d..d05e0ea 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/handler/HANamespaceHandler.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/handler/HANamespaceHandler.java
@@ -17,11 +17,8 @@
 
 package org.apache.shardingsphere.ha.spring.namespace.handler;
 
-import org.apache.shardingsphere.ha.spring.namespace.factorybean.ReplicaLoadBalanceAlgorithmFactoryBean;
 import org.apache.shardingsphere.ha.spring.namespace.parser.HARuleBeanDefinitionParser;
-import org.apache.shardingsphere.ha.spring.namespace.tag.LoadBalanceAlgorithmBeanDefinitionTag;
 import org.apache.shardingsphere.ha.spring.namespace.tag.HARuleBeanDefinitionTag;
-import org.apache.shardingsphere.spring.namespace.parser.ShardingSphereAlgorithmBeanDefinitionParser;
 import org.springframework.beans.factory.xml.NamespaceHandlerSupport;
 
 /**
@@ -32,6 +29,5 @@ public final class HANamespaceHandler extends NamespaceHandlerSupport {
     @Override
     public void init() {
         registerBeanDefinitionParser(HARuleBeanDefinitionTag.ROOT_TAG, new HARuleBeanDefinitionParser());
-        registerBeanDefinitionParser(LoadBalanceAlgorithmBeanDefinitionTag.ROOT_TAG, new ShardingSphereAlgorithmBeanDefinitionParser(ReplicaLoadBalanceAlgorithmFactoryBean.class));
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/parser/HARuleBeanDefinitionParser.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/parser/HARuleBeanDefinitionParser.java
index 37c2429..2b7687f 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/parser/HARuleBeanDefinitionParser.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/parser/HARuleBeanDefinitionParser.java
@@ -17,12 +17,9 @@
 
 package org.apache.shardingsphere.ha.spring.namespace.parser;
 
-import com.google.common.base.Splitter;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
-import org.apache.shardingsphere.ha.spring.namespace.factorybean.ReplicaLoadBalanceAlgorithmFactoryBean;
 import org.apache.shardingsphere.ha.spring.namespace.tag.HARuleBeanDefinitionTag;
-import org.apache.shardingsphere.spring.namespace.registry.ShardingSphereAlgorithmBeanRegistry;
 import org.springframework.beans.factory.config.BeanDefinition;
 import org.springframework.beans.factory.support.AbstractBeanDefinition;
 import org.springframework.beans.factory.support.BeanDefinitionBuilder;
@@ -32,7 +29,6 @@ import org.springframework.beans.factory.xml.ParserContext;
 import org.springframework.util.xml.DomUtils;
 import org.w3c.dom.Element;
 
-import java.util.Collection;
 import java.util.List;
 
 /**
@@ -44,7 +40,6 @@ public final class HARuleBeanDefinitionParser extends AbstractBeanDefinitionPars
     protected AbstractBeanDefinition parseInternal(final Element element, final ParserContext parserContext) {
         BeanDefinitionBuilder factory = BeanDefinitionBuilder.rootBeanDefinition(AlgorithmProvidedHARuleConfiguration.class);
         factory.addConstructorArgValue(parseHADataSourceRuleConfigurations(element));
-        factory.addConstructorArgValue(ShardingSphereAlgorithmBeanRegistry.getAlgorithmBeanReferences(parserContext, ReplicaLoadBalanceAlgorithmFactoryBean.class));
         return factory.getBeanDefinition();
     }
     
@@ -60,17 +55,6 @@ public final class HARuleBeanDefinitionParser extends AbstractBeanDefinitionPars
     private BeanDefinition parseHADataSourceRuleConfiguration(final Element element) {
         BeanDefinitionBuilder factory = BeanDefinitionBuilder.rootBeanDefinition(HADataSourceRuleConfiguration.class);
         factory.addConstructorArgValue(element.getAttribute(HARuleBeanDefinitionTag.HA_DATA_SOURCE_ID_ATTRIBUTE));
-        factory.addConstructorArgValue(element.getAttribute(HARuleBeanDefinitionTag.PRIMARY_DATA_SOURCE_NAME_ATTRIBUTE));
-        factory.addConstructorArgValue(parseReplicaDataSourcesRef(element));
-        factory.addConstructorArgValue(element.getAttribute(HARuleBeanDefinitionTag.LOAD_BALANCE_ALGORITHM_REF_ATTRIBUTE));
-        factory.addConstructorArgValue(element.getAttribute(HARuleBeanDefinitionTag.READ_WRITE_SPLIT_ATTRIBUTE));
         return factory.getBeanDefinition();
     }
-    
-    private Collection<String> parseReplicaDataSourcesRef(final Element element) {
-        List<String> replicaDataSources = Splitter.on(",").trimResults().splitToList(element.getAttribute(HARuleBeanDefinitionTag.REPLICA_DATA_SOURCE_NAMES_ATTRIBUTE));
-        Collection<String> result = new ManagedList<>(replicaDataSources.size());
-        result.addAll(replicaDataSources);
-        return result;
-    }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/HARuleBeanDefinitionTag.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/HARuleBeanDefinitionTag.java
index 080c445..ee63708 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/HARuleBeanDefinitionTag.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/HARuleBeanDefinitionTag.java
@@ -31,12 +31,4 @@ public final class HARuleBeanDefinitionTag {
     public static final String DATA_SOURCE_TAG = "data-source-rule";
     
     public static final String HA_DATA_SOURCE_ID_ATTRIBUTE = "id";
-    
-    public static final String PRIMARY_DATA_SOURCE_NAME_ATTRIBUTE = "primary-data-source-name";
-    
-    public static final String REPLICA_DATA_SOURCE_NAMES_ATTRIBUTE = "replica-data-source-names";
-    
-    public static final String LOAD_BALANCE_ALGORITHM_REF_ATTRIBUTE = "load-balance-algorithm-ref";
-    
-    public static final String READ_WRITE_SPLIT_ATTRIBUTE = "read-write-split";
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/LoadBalanceAlgorithmBeanDefinitionTag.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/LoadBalanceAlgorithmBeanDefinitionTag.java
deleted file mode 100644
index 30d3af7..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/java/org/apache/shardingsphere/ha/spring/namespace/tag/LoadBalanceAlgorithmBeanDefinitionTag.java
+++ /dev/null
@@ -1,30 +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.ha.spring.namespace.tag;
-
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
-
-/**
- * Load balance algorithm bean definition tag.
- */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
-public final class LoadBalanceAlgorithmBeanDefinitionTag {
-    
-    public static final String ROOT_TAG = "load-balance-algorithm";
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/resources/META-INF/namespace/ha.xsd b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/resources/META-INF/namespace/ha.xsd
index b1d84f9..3b1795d 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/resources/META-INF/namespace/ha.xsd
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/main/resources/META-INF/namespace/ha.xsd
@@ -35,20 +35,6 @@
     <xsd:element name="data-source-rule">
         <xsd:complexType>
             <xsd:attribute name="id" type="xsd:string" use="required" />
-            <xsd:attribute name="primary-data-source-name" type="xsd:string" use="required" />
-            <xsd:attribute name="replica-data-source-names" type="xsd:string" use="required" />
-            <xsd:attribute name="load-balance-algorithm-ref" type="xsd:string" />
-            <xsd:attribute name="read-write-split" type="xsd:boolean" />
-        </xsd:complexType>
-    </xsd:element>
-    
-    <xsd:element name="load-balance-algorithm">
-        <xsd:complexType>
-            <xsd:all>
-                <xsd:element ref="beans:props" minOccurs="0" />
-            </xsd:all>
-            <xsd:attribute name="id" type="xsd:string" use="required" />
-            <xsd:attribute name="type" type="xsd:string" />
         </xsd:complexType>
     </xsd:element>
 </xsd:schema>
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/ha-rule.yaml b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/ha-rule.yaml
index ddccee2..7072f50 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/ha-rule.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/ha-rule.yaml
@@ -24,8 +24,6 @@ rules:
           - ds_0
           - ds_1
           - ds_2
-        loadBalancerName: random
-        replicaQuery: false
         haTypeName: mgr
     haTypes:
       mgr:
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/registryCenter/data-ha-rule.yaml b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/registryCenter/data-ha-rule.yaml
index ddccee2..7072f50 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/registryCenter/data-ha-rule.yaml
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/resources/yaml/registryCenter/data-ha-rule.yaml
@@ -24,8 +24,6 @@ rules:
           - ds_0
           - ds_1
           - ds_2
-        loadBalancerName: random
-        replicaQuery: false
         haTypeName: mgr
     haTypes:
       mgr: