You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by pa...@apache.org on 2020/11/25 06:59:10 UTC

[shardingsphere] branch master updated: #7318, add SPI for MGR HA (#8315)

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

panjuan 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 5d29a47  #7318, add SPI for MGR HA (#8315)
5d29a47 is described below

commit 5d29a47c6133844289019d7aac137fc8e767d7ee
Author: Zhang Yonglun <zh...@apache.org>
AuthorDate: Wed Nov 25 14:58:56 2020 +0800

    #7318, add SPI for MGR HA (#8315)
    
    * #7318, add SPI for MGR HA
    
    * #7318, add SPI for MGR HA
    
    * #7318, for test cases
    
    * #7318, for test cases
    
    * #7318, for test cases
---
 .../ha/api/config/HARuleConfiguration.java         |   2 +
 .../HARuleConfiguration.java => spi/HAType.java}   |  39 ++++--
 .../shardingsphere/ha/algorithm/MGRHAType.java     | 153 +++++++++++++++++++++
 .../AlgorithmProvidedHARuleConfiguration.java      |   5 +-
 .../org/apache/shardingsphere/ha/rule/HARule.java  |  30 +++-
 .../biulder/AlgorithmProvidedHARuleBuilder.java    |  15 +-
 .../ha/rule/biulder/HARuleBuilder.java             |  15 +-
 .../ha/yaml/config/YamlHARuleConfiguration.java    |   6 +-
 .../swapper/HARuleConfigurationYamlSwapper.java    |   3 +-
 .../org.apache.shardingsphere.ha.spi.HAType        |  18 +++
 .../ha/fixture/TestHATypeFixture.java}             |  36 +++--
 .../apache/shardingsphere/ha/rule/HARuleTest.java  |  15 +-
 .../AlgorithmProvidedHARuleBuilderTest.java        |   2 +-
 .../ha/rule/biulder/HARuleBuilderTest.java         |   2 +-
 ...orithmProviderConfigurationYamlSwapperTest.java |   7 +-
 .../HARuleConfigurationYamlSwapperTest.java        |  11 +-
 .../org.apache.shardingsphere.ha.spi.HAType        |  18 +++
 .../src/test/resources/yaml/ha-rule.yaml           |   6 +
 .../ha/route/engine/HASQLRouterTest.java           |   6 +-
 .../ha/route/fixture/TestRouteHATypeFixture.java}  |  36 +++--
 .../org.apache.shardingsphere.ha.spi.HAType        |  18 +++
 .../ha/spring/namespace/HASpringNamespaceTest.java |  87 ------------
 .../META-INF/spring/ha-application-context.xml     |  36 -----
 .../src/test/resources/logback-test.xml            |  33 -----
 .../src/main/resources/conf/config-ha.yaml         |   5 +
 25 files changed, 388 insertions(+), 216 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 e568510..235b2bc 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
@@ -36,4 +36,6 @@ public final class HARuleConfiguration implements RuleConfiguration {
     private final Collection<HADataSourceRuleConfiguration> dataSources;
     
     private final Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers;
+    
+    private final ShardingSphereAlgorithmConfiguration haType;
 }
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/spi/HAType.java
similarity index 55%
copy from shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/api/config/HARuleConfiguration.java
copy to shardingsphere-features/shardingsphere-ha/shardingsphere-ha-api/src/main/java/org/apache/shardingsphere/ha/spi/HAType.java
index e568510..a57ea30 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/spi/HAType.java
@@ -15,25 +15,38 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.ha.api.config;
+package org.apache.shardingsphere.ha.spi;
 
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
-import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithm;
 
-import java.util.Collection;
+import javax.sql.DataSource;
+import java.sql.SQLException;
 import java.util.Map;
 
 /**
- * HA rule configuration.
+ * HA type.
  */
-@RequiredArgsConstructor
-@Getter
-public final class HARuleConfiguration implements RuleConfiguration {
+public interface HAType extends ShardingSphereAlgorithm {
     
-    private final Collection<HADataSourceRuleConfiguration> dataSources;
+    /**
+     * Check HA config.
+     *
+     * @param dataSourceMap Data source map
+     * @throws SQLException SQL Exception
+     */
+    void checkHAConfig(Map<String, DataSource> dataSourceMap) throws SQLException;
     
-    private final Map<String, ShardingSphereAlgorithmConfiguration> loadBalancers;
+    /**
+     * Update primary data source.
+     *
+     * @param dataSourceMap Data source map
+     */
+    void updatePrimaryDataSource(Map<String, DataSource> dataSourceMap);
+    
+    /**
+     * Periodical monitor.
+     *
+     * @param dataSourceMap Data source map
+     */
+    void periodicalMonitor(Map<String, DataSource> dataSourceMap);
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/MGRHAType.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/MGRHAType.java
new file mode 100644
index 0000000..0dc25d7
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/algorithm/MGRHAType.java
@@ -0,0 +1,153 @@
+/*
+ * 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.HAType;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
+
+import javax.sql.DataSource;
+import java.sql.Connection;
+import java.sql.ResultSet;
+import java.sql.SQLException;
+import java.sql.Statement;
+import java.util.Map;
+import java.util.Properties;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * MGR HA type.
+ */
+public final class MGRHAType implements HAType {
+    
+    private static final String PLUGIN_STATUS = "SELECT * FROM information_schema.PLUGINS WHERE PLUGIN_NAME='group_replication'";
+    
+    private static final String MEMBER_COUNT = "SELECT count(*) FROM performance_schema.replication_group_members";
+    
+    private static final String GROUP_NAME = "SELECT * FROM performance_schema.global_variables WHERE VARIABLE_NAME='group_replication_group_name'";
+    
+    private static final String SINGLE_PRIMARY = "SELECT * FROM performance_schema.global_variables WHERE VARIABLE_NAME='group_replication_single_primary_mode'";
+    
+    private String primaryDataSource;
+    
+    private ScheduledExecutorService scheduledExecutorService = Executors.newSingleThreadScheduledExecutor();
+    
+    @Getter
+    @Setter
+    private Properties props = new Properties();
+    
+    @Override
+    public void checkHAConfig(final Map<String, DataSource> dataSourceMap) throws SQLException {
+        try (Connection connection = dataSourceMap.get(primaryDataSource).getConnection();
+             Statement statement = connection.createStatement()) {
+            ResultSet resultSet = statement.executeQuery(PLUGIN_STATUS);
+            while (resultSet.next()) {
+                if (!"ACTIVE".equals(resultSet.getString("PLUGIN_STATUS"))) {
+                    throw new ShardingSphereException("MGR plugin is not active.");
+                }
+            }
+            resultSet.close();
+            resultSet = statement.executeQuery(MEMBER_COUNT);
+            while (resultSet.next()) {
+                if (Integer.parseInt(resultSet.getString(1)) < 1) {
+                    throw new ShardingSphereException("MGR member count < 1");
+                }
+            }
+            resultSet.close();
+            resultSet = statement.executeQuery(GROUP_NAME);
+            while (resultSet.next()) {
+                String serverGroupName = resultSet.getString("VARIABLE_VALUE");
+                String ruleGroupName = props.getProperty("groupName");
+                if (!serverGroupName.equals(ruleGroupName)) {
+                    throw new ShardingSphereException("MGR group name is not consistent\n" + "serverGroupName: " + serverGroupName
+                            + "\nruleGroupName: " + ruleGroupName);
+                }
+            }
+            resultSet.close();
+            resultSet = statement.executeQuery(SINGLE_PRIMARY);
+            while (resultSet.next()) {
+                if (!"ON".equals(resultSet.getString("VARIABLE_VALUE"))) {
+                    throw new ShardingSphereException("MGR is not in single primary mode");
+                }
+            }
+            resultSet.close();
+        }
+    }
+    
+    @Override
+    public void updatePrimaryDataSource(final Map<String, DataSource> dataSourceMap) {
+        String primary = queryPrimaryDataSource(dataSourceMap);
+        if (!"".equals(primary)) {
+            primaryDataSource = primary;
+        }
+    }
+    
+    private String queryPrimaryDataSource(final Map<String, DataSource> dataSourceMap) {
+        String result = "";
+        String urlResult = "";
+        for (Map.Entry<String, DataSource> entry : dataSourceMap.entrySet()) {
+            DataSource dataSource = entry.getValue();
+            String url = "";
+            String sql = "SELECT MEMBER_HOST, MEMBER_PORT FROM performance_schema.replication_group_members WHERE MEMBER_ID = "
+                    + "(SELECT VARIABLE_VALUE FROM performance_schema.global_status WHERE VARIABLE_NAME = 'group_replication_primary_member')";
+            try (Connection connection = dataSource.getConnection();
+                 Statement statement = connection.createStatement();
+                 ResultSet resultSet = statement.executeQuery(sql)) {
+                while (resultSet.next()) {
+                    url = resultSet.getString("MEMBER_HOST");
+                    url += ":";
+                    url += resultSet.getString("MEMBER_PORT");
+                }
+                // CHECKSTYLE:OFF
+            } catch (final Exception ex) {
+                // CHECKSTYLE:ON
+            }
+            if (null != url && !"".equals(url) && !"".equals(urlResult) && !urlResult.equals(url)) {
+                return result;
+            }
+            urlResult = url;
+        }
+        for (Map.Entry<String, DataSource> entry : dataSourceMap.entrySet()) {
+            DataSource dataSource = entry.getValue();
+            try (Connection connection = dataSource.getConnection()) {
+                if (connection.getMetaData().getURL().contains(urlResult)) {
+                    result = entry.getKey();
+                    break;
+                }
+                // CHECKSTYLE:OFF
+            } catch (final Exception ex) {
+                // CHECKSTYLE:ON
+            }
+        }
+        return result;
+    }
+    
+    @Override
+    public void periodicalMonitor(final Map<String, DataSource> dataSourceMap) {
+        Runnable runnable = () -> updatePrimaryDataSource(dataSourceMap);
+        scheduledExecutorService.scheduleAtFixedRate(runnable, 0, Integer.parseInt(props.getProperty("keepAliveSeconds")), TimeUnit.SECONDS);
+    }
+    
+    @Override
+    public String getType() {
+        return "MGR";
+    }
+}
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 e90a0fe..4f348d6 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
@@ -21,9 +21,10 @@ import lombok.AllArgsConstructor;
 import lombok.Getter;
 import lombok.NoArgsConstructor;
 import lombok.Setter;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
 import org.apache.shardingsphere.ha.spi.ReplicaLoadBalanceAlgorithm;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 
 import java.util.Collection;
 import java.util.LinkedHashMap;
@@ -42,4 +43,6 @@ public final class AlgorithmProvidedHARuleConfiguration implements RuleConfigura
     private Collection<HADataSourceRuleConfiguration> dataSources = new LinkedList<>();
     
     private Map<String, ReplicaLoadBalanceAlgorithm> loadBalanceAlgorithms = new LinkedHashMap<>();
+    
+    private ShardingSphereAlgorithmConfiguration haType;
 }
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 69a900a..3fa84ec 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
@@ -19,7 +19,10 @@ package org.apache.shardingsphere.ha.rule;
 
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
+import org.apache.shardingsphere.ha.spi.HAType;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmFactory;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.rule.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.type.StatusContainedRule;
 import org.apache.shardingsphere.infra.rule.event.RuleChangedEvent;
@@ -31,6 +34,8 @@ 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;
 import java.util.Collection;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -45,14 +50,17 @@ public final class HARule implements DataSourceContainedRule, StatusContainedRul
     
     static {
         ShardingSphereServiceLoader.register(ReplicaLoadBalanceAlgorithm.class);
+        ShardingSphereServiceLoader.register(HAType.class);
     }
     
     private final Map<String, ReplicaLoadBalanceAlgorithm> loadBalancers = new LinkedHashMap<>();
     
     private final Map<String, HADataSourceRule> dataSourceRules;
     
-    public HARule(final HARuleConfiguration config) {
+    public HARule(final HARuleConfiguration config, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap) {
         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)));
         dataSourceRules = new HashMap<>(config.getDataSources().size(), 1);
         for (HADataSourceRuleConfiguration each : config.getDataSources()) {
@@ -61,10 +69,20 @@ public final class HARule implements DataSourceContainedRule, StatusContainedRul
                     ? TypedSPIRegistry.getRegisteredService(ReplicaLoadBalanceAlgorithm.class) : loadBalancers.get(each.getLoadBalancerName());
             dataSourceRules.put(each.getName(), new HADataSourceRule(each, loadBalanceAlgorithm));
         }
+        HAType haType = TypedSPIRegistry.getRegisteredService(HAType.class, config.getHaType().getType(), config.getHaType().getProps());
+        try {
+            haType.updatePrimaryDataSource(dataSourceMap);
+            haType.checkHAConfig(dataSourceMap);
+            haType.periodicalMonitor(dataSourceMap);
+        } catch (final SQLException ex) {
+            throw new ShardingSphereException(ex);
+        }
     }
     
-    public HARule(final AlgorithmProvidedHARuleConfiguration config) {
+    public HARule(final AlgorithmProvidedHARuleConfiguration config, final DatabaseType databaseType, final Map<String, DataSource> dataSourceMap) {
         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()) {
@@ -73,6 +91,14 @@ public final class HARule implements DataSourceContainedRule, StatusContainedRul
                     ? TypedSPIRegistry.getRegisteredService(ReplicaLoadBalanceAlgorithm.class) : loadBalancers.get(each.getLoadBalancerName());
             dataSourceRules.put(each.getName(), new HADataSourceRule(each, loadBalanceAlgorithm));
         }
+        HAType haType = TypedSPIRegistry.getRegisteredService(HAType.class, config.getHaType().getType(), config.getHaType().getProps());
+        try {
+            haType.updatePrimaryDataSource(dataSourceMap);
+            haType.checkHAConfig(dataSourceMap);
+            haType.periodicalMonitor(dataSourceMap);
+        } catch (final SQLException ex) {
+            throw new ShardingSphereException(ex);
+        }
     }
     
     /**
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilder.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilder.java
index dc5df97..76c7bdc 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/AlgorithmProvidedHARuleBuilder.java
@@ -17,19 +17,30 @@
 
 package org.apache.shardingsphere.ha.rule.biulder;
 
+import lombok.Setter;
 import org.apache.shardingsphere.ha.algorithm.config.AlgorithmProvidedHARuleConfiguration;
 import org.apache.shardingsphere.ha.constant.HAOrder;
 import org.apache.shardingsphere.ha.rule.HARule;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.aware.ResourceAware;
+
+import javax.sql.DataSource;
+import java.util.Map;
 
 /**
  * Algorithm provided HA rule builder.
  */
-public final class AlgorithmProvidedHARuleBuilder implements ShardingSphereRuleBuilder<HARule, AlgorithmProvidedHARuleConfiguration> {
+@Setter
+public final class AlgorithmProvidedHARuleBuilder implements ShardingSphereRuleBuilder<HARule, AlgorithmProvidedHARuleConfiguration>, ResourceAware {
+    
+    private DatabaseType databaseType;
+    
+    private Map<String, DataSource> dataSourceMap;
     
     @Override
     public HARule build(final AlgorithmProvidedHARuleConfiguration ruleConfig) {
-        return new HARule(ruleConfig);
+        return new HARule(ruleConfig, databaseType, dataSourceMap);
     }
     
     @Override
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
index 74a4504..0f0f294 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
@@ -17,19 +17,30 @@
 
 package org.apache.shardingsphere.ha.rule.biulder;
 
+import lombok.Setter;
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.constant.HAOrder;
 import org.apache.shardingsphere.ha.rule.HARule;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import org.apache.shardingsphere.infra.rule.builder.aware.ResourceAware;
+
+import javax.sql.DataSource;
+import java.util.Map;
 
 /**
  * HA rule builder.
  */
-public final class HARuleBuilder implements ShardingSphereRuleBuilder<HARule, HARuleConfiguration> {
+@Setter
+public final class HARuleBuilder implements ShardingSphereRuleBuilder<HARule, HARuleConfiguration>, ResourceAware {
+    
+    private DatabaseType databaseType;
+    
+    private Map<String, DataSource> dataSourceMap;
     
     @Override
     public HARule build(final HARuleConfiguration ruleConfig) {
-        return new HARule(ruleConfig);
+        return new HARule(ruleConfig, databaseType, dataSourceMap);
     }
     
     @Override
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 c05f452..1cb3f33 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
@@ -19,10 +19,10 @@ package org.apache.shardingsphere.ha.yaml.config;
 
 import lombok.Getter;
 import lombok.Setter;
-import org.apache.shardingsphere.infra.yaml.config.YamlRuleConfiguration;
-import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.yaml.config.rule.YamlHADataSourceRuleConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.YamlRuleConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
 
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -38,6 +38,8 @@ public final class YamlHARuleConfiguration implements YamlRuleConfiguration {
     
     private Map<String, YamlShardingSphereAlgorithmConfiguration> loadBalancers = new LinkedHashMap<>();
     
+    private YamlShardingSphereAlgorithmConfiguration haType;
+    
     @Override
     public Class<HARuleConfiguration> getRuleConfigurationType() {
         return HARuleConfiguration.class;
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 8421544..3d533e3 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
@@ -72,7 +72,8 @@ public final class HARuleConfigurationYamlSwapper
         if (null != yamlConfig.getLoadBalancers()) {
             yamlConfig.getLoadBalancers().forEach((key, value) -> loadBalancers.put(key, algorithmSwapper.swapToObject(value)));
         }
-        return new HARuleConfiguration(dataSources, loadBalancers);
+        ShardingSphereAlgorithmConfiguration haType = algorithmSwapper.swapToObject(yamlConfig.getHaType());
+        return new HARuleConfiguration(dataSources, loadBalancers, haType);
     }
     
     private HADataSourceRuleConfiguration swapToObject(final String name, final YamlHADataSourceRuleConfiguration yamlDataSourceRuleConfig) {
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
new file mode 100644
index 0000000..5e204d4
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
@@ -0,0 +1,18 @@
+#
+# 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.MGRHAType
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/fixture/TestHATypeFixture.java
similarity index 56%
copy from shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
copy to shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/fixture/TestHATypeFixture.java
index 74a4504..7616f16 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/java/org/apache/shardingsphere/ha/fixture/TestHATypeFixture.java
@@ -15,30 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.ha.rule.biulder;
+package org.apache.shardingsphere.ha.fixture;
 
-import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
-import org.apache.shardingsphere.ha.constant.HAOrder;
-import org.apache.shardingsphere.ha.rule.HARule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.ha.spi.HAType;
+
+import javax.sql.DataSource;
+import java.util.Map;
+import java.util.Properties;
 
 /**
- * HA rule builder.
+ * Test HA type.
  */
-public final class HARuleBuilder implements ShardingSphereRuleBuilder<HARule, HARuleConfiguration> {
+@Getter
+@Setter
+public final class TestHATypeFixture implements HAType {
+    
+    private Properties props = new Properties();
+    
+    @Override
+    public void checkHAConfig(final Map<String, DataSource> dataSourceMap) {
+    }
     
     @Override
-    public HARule build(final HARuleConfiguration ruleConfig) {
-        return new HARule(ruleConfig);
+    public void updatePrimaryDataSource(final Map<String, DataSource> dataSourceMap) {
+
     }
     
     @Override
-    public int getOrder() {
-        return HAOrder.ORDER;
+    public void periodicalMonitor(final Map<String, DataSource> dataSourceMap) {
     }
     
     @Override
-    public Class<HARuleConfiguration> getTypeClass() {
-        return HARuleConfiguration.class;
+    public String getType() {
+        return "Test";
     }
 }
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 74b9298..733155b 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
@@ -18,12 +18,14 @@
 package org.apache.shardingsphere.ha.rule;
 
 import com.google.common.collect.ImmutableMap;
-import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
+import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
 import org.junit.Test;
 
+import javax.sql.DataSource;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -34,12 +36,15 @@ import java.util.Properties;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public final class HARuleTest {
     
+    private final Map<String, DataSource> dataSourceMap = Collections.singletonMap("ds", mock(DataSource.class));
+    
     @Test(expected = IllegalArgumentException.class)
     public void assertNewWithEmptyDataSourceRule() {
-        new HARule(new HARuleConfiguration(Collections.emptyList(), Collections.emptyMap()));
+        new HARule(new HARuleConfiguration(Collections.emptyList(), Collections.emptyMap(), mock(ShardingSphereAlgorithmConfiguration.class)), mock(DatabaseType.class), dataSourceMap);
     }
     
     @Test
@@ -58,7 +63,9 @@ public final class HARuleTest {
         HADataSourceRuleConfiguration config =
                 new HADataSourceRuleConfiguration("test_pr", "primary_ds", Arrays.asList("replica_ds_0", "replica_ds_1"), "random", true);
         return new HARule(new HARuleConfiguration(
-                Collections.singleton(config), ImmutableMap.of("random", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties()))));
+                Collections.singleton(config), ImmutableMap.of("random", new ShardingSphereAlgorithmConfiguration("RANDOM", new Properties())),
+                new ShardingSphereAlgorithmConfiguration("Test", new Properties())),
+                mock(DatabaseType.class), dataSourceMap);
     }
     
     private void assertDataSourceRule(final HADataSourceRule actual) {
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 670073f..008b2bd 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
@@ -39,7 +39,7 @@ public final class AlgorithmProvidedHARuleBuilderTest {
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
-    @Test
+    @Test(expected = IllegalArgumentException.class)
     public void assertBuild() {
         AlgorithmProvidedHARuleConfiguration algorithmProvidedRuleConfig = mock(AlgorithmProvidedHARuleConfiguration.class);
         HADataSourceRuleConfiguration ruleConfig = new HADataSourceRuleConfiguration(
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 01b107e..b96c15b 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
@@ -39,7 +39,7 @@ public final class HARuleBuilderTest {
     }
     
     @SuppressWarnings({"rawtypes", "unchecked"})
-    @Test
+    @Test(expected = IllegalArgumentException.class)
     public void assertBuild() {
         HARuleConfiguration ruleConfig = mock(HARuleConfiguration.class);
         HADataSourceRuleConfiguration dataSourceRuleConfig = new HADataSourceRuleConfiguration(
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 18300a8..470dcfa 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,14 +18,16 @@
 package org.apache.shardingsphere.ha.yaml.swapper;
 
 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.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.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.junit.Test;
 
 import java.util.Collections;
+import java.util.Properties;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
@@ -87,6 +89,7 @@ public final class HARuleAlgorithmProviderConfigurationYamlSwapperTest {
         HADataSourceRuleConfiguration ruleConfig = new HADataSourceRuleConfiguration("name", "primaryDataSourceName",
                 Collections.singletonList("replicaDataSourceName"), "loadBalancerName", true);
         return swapper.swapToYamlConfiguration(
-                new AlgorithmProvidedHARuleConfiguration(Collections.singletonList(ruleConfig), ImmutableMap.of("name", new RandomReplicaLoadBalanceAlgorithm())));
+                new AlgorithmProvidedHARuleConfiguration(Collections.singletonList(ruleConfig), ImmutableMap.of("name", new RandomReplicaLoadBalanceAlgorithm()),
+                        new ShardingSphereAlgorithmConfiguration("name", new Properties())));
     }
 }
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 a0055af..d693289 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
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.ha.yaml.swapper;
 import com.google.common.collect.ImmutableMap;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapper;
 import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
 import org.apache.shardingsphere.ha.api.config.rule.HADataSourceRuleConfiguration;
@@ -38,6 +39,7 @@ import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertNull;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.mock;
 
 public final class HARuleConfigurationYamlSwapperTest {
     
@@ -52,7 +54,8 @@ public final class HARuleConfigurationYamlSwapperTest {
         HADataSourceRuleConfiguration dataSourceConfig =
                 new HADataSourceRuleConfiguration("ds", "primary", Collections.singletonList("replica"), "roundRobin", true);
         YamlHARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToYamlConfiguration(new HARuleConfiguration(
-                Collections.singleton(dataSourceConfig), ImmutableMap.of("roundRobin", new ShardingSphereAlgorithmConfiguration("ROUND_ROBIN", new Properties()))));
+                Collections.singleton(dataSourceConfig), ImmutableMap.of("roundRobin", new ShardingSphereAlgorithmConfiguration("ROUND_ROBIN", new Properties())),
+                mock(ShardingSphereAlgorithmConfiguration.class)));
         assertThat(actual.getDataSources().get("ds").getName(), is("ds"));
         assertThat(actual.getDataSources().get("ds").getPrimaryDataSourceName(), is("primary"));
         assertThat(actual.getDataSources().get("ds").getReplicaDataSourceNames(), is(Collections.singletonList("replica")));
@@ -63,7 +66,7 @@ public final class HARuleConfigurationYamlSwapperTest {
     public void assertSwapToYamlWithoutLoadBalanceAlgorithm() {
         HADataSourceRuleConfiguration dataSourceConfig = new HADataSourceRuleConfiguration("ds", "primary", Collections.singletonList("replica"), null, true);
         YamlHARuleConfiguration actual = getHARuleConfigurationYamlSwapper().swapToYamlConfiguration(
-                new HARuleConfiguration(Collections.singleton(dataSourceConfig), Collections.emptyMap()));
+                new HARuleConfiguration(Collections.singleton(dataSourceConfig), Collections.emptyMap(), mock(ShardingSphereAlgorithmConfiguration.class)));
         assertThat(actual.getDataSources().get("ds").getName(), is("ds"));
         assertThat(actual.getDataSources().get("ds").getPrimaryDataSourceName(), is("primary"));
         assertThat(actual.getDataSources().get("ds").getReplicaDataSourceNames(), is(Collections.singletonList("replica")));
@@ -93,6 +96,10 @@ public final class HARuleConfigurationYamlSwapperTest {
         result.getDataSources().get("ha_ds").setName("ha_ds");
         result.getDataSources().get("ha_ds").setPrimaryDataSourceName("primary_ds");
         result.getDataSources().get("ha_ds").setReplicaDataSourceNames(Arrays.asList("replica_ds_0", "replica_ds_1"));
+        YamlShardingSphereAlgorithmConfiguration haType = new YamlShardingSphereAlgorithmConfiguration();
+        haType.setType("name");
+        haType.setProps(new Properties());
+        result.setHaType(haType);
         return result;
     }
     
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
new file mode 100644
index 0000000..15e71e6
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
@@ -0,0 +1,18 @@
+#
+# 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.fixture.TestHATypeFixture
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 6c3e888..665e8f5 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
@@ -69,3 +69,9 @@ rules:
       type: ROUND_ROBIN
     random:
       type: RANDOM
+
+  haType:
+    type: MGR
+    props:
+      groupName: 92504d5b-6dec-11e8-91ea-246e9612aaf1
+      keepAliveSeconds: '5'
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 ca6c7fb..28da1df 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
@@ -23,7 +23,9 @@ 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;
+import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationProperties;
+import org.apache.shardingsphere.infra.database.type.DatabaseType;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
 import org.apache.shardingsphere.infra.metadata.rule.ShardingSphereRuleMetaData;
@@ -45,6 +47,7 @@ import org.junit.runner.RunWith;
 import org.mockito.Mock;
 import org.mockito.junit.MockitoJUnitRunner;
 
+import javax.sql.DataSource;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.Optional;
@@ -81,7 +84,8 @@ public final class HASQLRouterTest {
     @Before
     public void setUp() {
         rule = new HARule(new HARuleConfiguration(Collections.singleton(
-                new HADataSourceRuleConfiguration(DATASOURCE_NAME, PRIMARY_DATASOURCE, Collections.singletonList(REPLICA_DATASOURCE), null, true)), Collections.emptyMap()));
+                new HADataSourceRuleConfiguration(DATASOURCE_NAME, PRIMARY_DATASOURCE, Collections.singletonList(REPLICA_DATASOURCE), null, true)),
+                Collections.emptyMap(), new ShardingSphereAlgorithmConfiguration("TestRoute", new Properties())), mock(DatabaseType.class), Collections.singletonMap("ds", mock(DataSource.class)));
         sqlRouter = (HASQLRouter) OrderedSPIRegistry.getRegisteredServices(Collections.singleton(rule), SQLRouter.class).get(rule);
     }
     
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/fixture/TestRouteHATypeFixture.java
similarity index 55%
copy from shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
copy to shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/fixture/TestRouteHATypeFixture.java
index 74a4504..c0f88df 100644
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-common/src/main/java/org/apache/shardingsphere/ha/rule/biulder/HARuleBuilder.java
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/java/org/apache/shardingsphere/ha/route/fixture/TestRouteHATypeFixture.java
@@ -15,30 +15,40 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.ha.rule.biulder;
+package org.apache.shardingsphere.ha.route.fixture;
 
-import org.apache.shardingsphere.ha.api.config.HARuleConfiguration;
-import org.apache.shardingsphere.ha.constant.HAOrder;
-import org.apache.shardingsphere.ha.rule.HARule;
-import org.apache.shardingsphere.infra.rule.builder.ShardingSphereRuleBuilder;
+import lombok.Getter;
+import lombok.Setter;
+import org.apache.shardingsphere.ha.spi.HAType;
+
+import javax.sql.DataSource;
+import java.util.Map;
+import java.util.Properties;
 
 /**
- * HA rule builder.
+ * Test HA type.
  */
-public final class HARuleBuilder implements ShardingSphereRuleBuilder<HARule, HARuleConfiguration> {
+@Getter
+@Setter
+public final class TestRouteHATypeFixture implements HAType {
+    
+    private Properties props = new Properties();
+    
+    @Override
+    public void checkHAConfig(final Map<String, DataSource> dataSourceMap) {
+    }
     
     @Override
-    public HARule build(final HARuleConfiguration ruleConfig) {
-        return new HARule(ruleConfig);
+    public void updatePrimaryDataSource(final Map<String, DataSource> dataSourceMap) {
+
     }
     
     @Override
-    public int getOrder() {
-        return HAOrder.ORDER;
+    public void periodicalMonitor(final Map<String, DataSource> dataSourceMap) {
     }
     
     @Override
-    public Class<HARuleConfiguration> getTypeClass() {
-        return HARuleConfiguration.class;
+    public String getType() {
+        return "TestRoute";
     }
 }
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
new file mode 100644
index 0000000..a0e7a7f
--- /dev/null
+++ b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-route/src/test/resources/META-INF/services/org.apache.shardingsphere.ha.spi.HAType
@@ -0,0 +1,18 @@
+#
+# 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.route.fixture.TestRouteHATypeFixture
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/java/org/apache/shardingsphere/ha/spring/namespace/HASpringNamespaceTest.java b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/java/org/apache/shardingsphere/ha/spring/namespace/HASpringNamespaceTest.java
deleted file mode 100644
index 04d86ad..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/java/org/apache/shardingsphere/ha/spring/namespace/HASpringNamespaceTest.java
+++ /dev/null
@@ -1,87 +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;
-
-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.spi.ReplicaLoadBalanceAlgorithm;
-import org.junit.Test;
-import org.springframework.test.context.ContextConfiguration;
-import org.springframework.test.context.junit4.AbstractJUnit4SpringContextTests;
-
-import javax.annotation.Resource;
-import java.util.Arrays;
-import java.util.Map;
-
-import static org.hamcrest.CoreMatchers.instanceOf;
-import static org.hamcrest.CoreMatchers.is;
-import static org.junit.Assert.assertThat;
-import static org.junit.Assert.assertTrue;
-
-@ContextConfiguration(locations = "classpath:META-INF/spring/ha-application-context.xml")
-public final class HASpringNamespaceTest extends AbstractJUnit4SpringContextTests {
-    
-    @Resource
-    private ReplicaLoadBalanceAlgorithm randomLoadbalancer;
-    
-    @Resource
-    private AlgorithmProvidedHARuleConfiguration defaultRule;
-    
-    @Resource
-    private AlgorithmProvidedHARuleConfiguration randomRule;
-    
-    @Test
-    public void assertRandomLoadbalancer() {
-        assertThat(randomLoadbalancer.getType(), is("RANDOM"));
-    }
-    
-    @Test
-    public void assertDefaultDataSource() {
-        assertLoadBalancers(defaultRule.getLoadBalanceAlgorithms());
-        assertThat(defaultRule.getDataSources().size(), is(1));
-        assertDefaultDataSourceRule(defaultRule.getDataSources().iterator().next());
-    }
-    
-    private void assertLoadBalancers(final Map<String, ReplicaLoadBalanceAlgorithm> loadBalances) {
-        assertThat(loadBalances.size(), is(1));
-        assertThat(loadBalances.get("randomLoadbalancer"), instanceOf(RandomReplicaLoadBalanceAlgorithm.class));
-    }
-    
-    private void assertDefaultDataSourceRule(final HADataSourceRuleConfiguration dataSourceRuleConfig) {
-        assertThat(dataSourceRuleConfig.getName(), is("default_ds"));
-        assertThat(dataSourceRuleConfig.getPrimaryDataSourceName(), is("primary_ds"));
-        assertThat(dataSourceRuleConfig.getReplicaDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
-        assertThat(dataSourceRuleConfig.getLoadBalancerName(), is(""));
-    }
-    
-    @Test
-    public void assertRandomDataSource() {
-        assertLoadBalancers(randomRule.getLoadBalanceAlgorithms());
-        assertThat(randomRule.getDataSources().size(), is(1));
-        assertRandomDataSourceRule(randomRule.getDataSources().iterator().next());
-    }
-    
-    private void assertRandomDataSourceRule(final HADataSourceRuleConfiguration dataSourceRuleConfig) {
-        assertThat(dataSourceRuleConfig.getName(), is("random_ds"));
-        assertThat(dataSourceRuleConfig.getPrimaryDataSourceName(), is("primary_ds"));
-        assertThat(dataSourceRuleConfig.getReplicaDataSourceNames(), is(Arrays.asList("replica_ds_0", "replica_ds_1")));
-        assertThat(dataSourceRuleConfig.getLoadBalancerName(), is("randomLoadbalancer"));
-        assertTrue(dataSourceRuleConfig.getReadWriteSplit());
-    }
-}
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/META-INF/spring/ha-application-context.xml b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/META-INF/spring/ha-application-context.xml
deleted file mode 100644
index 85368eb..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/META-INF/spring/ha-application-context.xml
+++ /dev/null
@@ -1,36 +0,0 @@
-<?xml version="1.0" encoding="UTF-8"?>
-<!--
-  ~ 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.
-  -->
-
-<beans xmlns="http://www.springframework.org/schema/beans"
-       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:ha="http://shardingsphere.apache.org/schema/shardingsphere/ha"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans
-                           http://www.springframework.org/schema/beans/spring-beans.xsd 
-                           http://shardingsphere.apache.org/schema/shardingsphere/ha
-                           http://shardingsphere.apache.org/schema/shardingsphere/ha/ha.xsd
-                           ">
-    <ha:load-balance-algorithm id="randomLoadbalancer" type="RANDOM" />
-    
-    <ha:rule id="defaultRule">
-        <ha:data-source-rule id="default_ds" primary-data-source-name="primary_ds" replica-data-source-names="replica_ds_0, replica_ds_1" />
-    </ha:rule>
-    
-    <ha:rule id="randomRule">
-        <ha:data-source-rule id="random_ds" primary-data-source-name="primary_ds" replica-data-source-names="replica_ds_0, replica_ds_1" load-balance-algorithm-ref="randomLoadbalancer" read-write-split="true"/>
-    </ha:rule>
-</beans>
diff --git a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/logback-test.xml b/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/logback-test.xml
deleted file mode 100644
index 7e7a77c..0000000
--- a/shardingsphere-features/shardingsphere-ha/shardingsphere-ha-spring/shardingsphere-ha-spring-namespace/src/test/resources/logback-test.xml
+++ /dev/null
@@ -1,33 +0,0 @@
-<?xml version="1.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.
-  -->
-
-<configuration>
-    <appender name="console" class="ch.qos.logback.core.ConsoleAppender">
-        <encoder>
-            <pattern>[%-5level] %d{HH:mm:ss.SSS} [%thread] %logger{36} - %msg%n</pattern>
-        </encoder>
-    </appender>
-    <logger name="org.apache.shardingsphere" level="warn" additivity="false">
-        <appender-ref ref="console" />
-    </logger>
-    
-    <root>
-        <level value="warn" />
-        <appender-ref ref="console" />
-    </root>
-</configuration> 
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-ha.yaml b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-ha.yaml
index 11ebb4b..df4ad3e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-ha.yaml
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/resources/conf/config-ha.yaml
@@ -116,3 +116,8 @@
 #        - replica_ds_0
 #        - replica_ds_1
 #      readWriteSplit: false
+#  haType:
+#    type: MGR
+#    props:
+#      groupName: 92504d5b-6dec-11e8-91ea-246e9612aaf1
+#      keepAliveSeconds: '5'