You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by zh...@apache.org on 2020/09/07 05:18:27 UTC

[shardingsphere] branch master updated: Add BootstrapInitializer and simplify bootstrap of proxy (#7286)

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

zhangyonglun 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 3ce2c99  Add BootstrapInitializer and simplify bootstrap of proxy (#7286)
3ce2c99 is described below

commit 3ce2c995b299c2345949c9a2f80251dbd0b3428a
Author: Liang Zhang <te...@163.com>
AuthorDate: Mon Sep 7 13:14:22 2020 +0800

    Add BootstrapInitializer and simplify bootstrap of proxy (#7286)
    
    * Add BootstrapInitializer
    
    * Revise javadoc
    
    * Refactor GovernanceBootstrap
    
    * Add AbstractBootstrapInitializer
    
    * Simplify Bootstrap
    
    * Remove GovernanceBootstrap
---
 .../org/apache/shardingsphere/proxy/Bootstrap.java | 80 ++--------------------
 .../proxy/init/BootstrapInitializer.java           | 37 ++++++++++
 .../impl/AbstractBootstrapInitializer.java}        | 63 +++++------------
 .../impl/GovernanceBootstrapInitializer.java}      | 48 ++++++++-----
 .../init/impl/StandardBootstrapInitializer.java    | 45 ++++++++++++
 .../impl/GovernanceBootstrapInitializerTest.java}  | 18 ++---
 6 files changed, 147 insertions(+), 144 deletions(-)

diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
index a57e69c..958404e 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
@@ -19,43 +19,21 @@ package org.apache.shardingsphere.proxy;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerInfo;
 import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
-import org.apache.shardingsphere.governance.context.schema.GovernanceSchemaContexts;
-import org.apache.shardingsphere.governance.context.transaction.GovernanceTransactionContexts;
-import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
-import org.apache.shardingsphere.infra.context.SchemaContext;
-import org.apache.shardingsphere.infra.context.SchemaContexts;
-import org.apache.shardingsphere.infra.context.SchemaContextsBuilder;
 import org.apache.shardingsphere.proxy.arg.BootstrapArguments;
-import org.apache.shardingsphere.proxy.backend.schema.ProxyDataSourceContext;
-import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
-import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.ProxyConfigurationLoader;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
-import org.apache.shardingsphere.proxy.config.yaml.swapper.YamlProxyConfigurationSwapper;
-import org.apache.shardingsphere.proxy.db.DatabaseServerInfo;
-import org.apache.shardingsphere.proxy.frontend.bootstrap.ShardingSphereProxy;
-import org.apache.shardingsphere.proxy.governance.GovernanceBootstrap;
-import org.apache.shardingsphere.tracing.opentracing.OpenTracingTracer;
-import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
-import org.apache.shardingsphere.transaction.context.TransactionContexts;
-import org.apache.shardingsphere.transaction.context.impl.StandardTransactionContexts;
+import org.apache.shardingsphere.proxy.init.BootstrapInitializer;
+import org.apache.shardingsphere.proxy.init.impl.GovernanceBootstrapInitializer;
+import org.apache.shardingsphere.proxy.init.impl.StandardBootstrapInitializer;
 
-import javax.sql.DataSource;
 import java.io.IOException;
 import java.sql.SQLException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Optional;
 
 /**
  * ShardingSphere-Proxy Bootstrap.
  */
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
-@Slf4j
 public final class Bootstrap {
     
     /**
@@ -69,56 +47,10 @@ public final class Bootstrap {
         BootstrapArguments bootstrapArgs = new BootstrapArguments(args);
         int port = bootstrapArgs.getPort();
         YamlProxyConfiguration yamlConfig = ProxyConfigurationLoader.load(bootstrapArgs.getConfigurationPath());
-        if (null == yamlConfig.getServerConfiguration().getGovernance()) {
-            init(new YamlProxyConfigurationSwapper().swap(yamlConfig), port, null);
-        } else {
-            GovernanceFacade governanceFacade = new GovernanceFacade();
-            init(new GovernanceBootstrap(governanceFacade).init(yamlConfig), port, governanceFacade);
-        }
+        createBootstrapInitializer(yamlConfig).init(yamlConfig, port);
     }
     
-    private static void init(final ProxyConfiguration proxyConfig, final int port, final GovernanceFacade governanceFacade) throws SQLException {
-        SchemaContexts schemaContexts = createSchemaContexts(proxyConfig);
-        TransactionContexts transactionContexts = createTransactionContexts(schemaContexts);
-        if (null != governanceFacade) {
-            schemaContexts = new GovernanceSchemaContexts(schemaContexts, governanceFacade);
-            transactionContexts = new GovernanceTransactionContexts(transactionContexts);
-        }
-        ProxySchemaContexts.getInstance().init(schemaContexts, transactionContexts);
-        initOpenTracing();
-        setDatabaseServerInfo();
-        ShardingSphereProxy.getInstance().start(port);
-    }
-    
-    private static SchemaContexts createSchemaContexts(final ProxyConfiguration proxyConfig) throws SQLException {
-        ProxyDataSourceContext dataSourceContext = new ProxyDataSourceContext(proxyConfig.getSchemaDataSources());
-        SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(
-                dataSourceContext.getDatabaseType(), dataSourceContext.getDataSourcesMap(), proxyConfig.getSchemaRules(), proxyConfig.getAuthentication(), proxyConfig.getProps());
-        return schemaContextsBuilder.build();
-    }
-    
-    private static TransactionContexts createTransactionContexts(final SchemaContexts schemaContexts) {
-        Map<String, ShardingTransactionManagerEngine> transactionManagerEngines = new HashMap<>(schemaContexts.getSchemaContexts().size(), 1);
-        for (Entry<String, SchemaContext> entry : schemaContexts.getSchemaContexts().entrySet()) {
-            ShardingTransactionManagerEngine engine = new ShardingTransactionManagerEngine();
-            engine.init(schemaContexts.getDatabaseType(), entry.getValue().getSchema().getDataSources());
-            transactionManagerEngines.put(entry.getKey(), engine);
-        }
-        return new StandardTransactionContexts(transactionManagerEngines);
-    }
-    
-    private static void initOpenTracing() {
-        if (ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_OPENTRACING_ENABLED)) {
-            OpenTracingTracer.init();
-        }
-    }
-    
-    private static void setDatabaseServerInfo() {
-        Optional<DataSource> dataSourceSample = ProxySchemaContexts.getInstance().getDataSourceSample();
-        if (dataSourceSample.isPresent()) {
-            DatabaseServerInfo databaseServerInfo = new DatabaseServerInfo(dataSourceSample.get());
-            log.info(databaseServerInfo.toString());
-            MySQLServerInfo.setServerVersion(databaseServerInfo.getDatabaseVersion());
-        }
+    private static BootstrapInitializer createBootstrapInitializer(final YamlProxyConfiguration yamlConfig) {
+        return null == yamlConfig.getServerConfiguration().getGovernance() ? new StandardBootstrapInitializer() : new GovernanceBootstrapInitializer(new GovernanceFacade());
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/BootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/BootstrapInitializer.java
new file mode 100644
index 0000000..2910df7
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/BootstrapInitializer.java
@@ -0,0 +1,37 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.init;
+
+import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
+
+import java.sql.SQLException;
+
+/**
+ * Bootstrap initializer.
+ */
+public interface BootstrapInitializer {
+    
+    /**
+     * Initialize.
+     * 
+     * @param yamlConfig YAML proxy configuration
+     * @param port port
+     * @throws SQLException SQL exception
+     */
+    void init(YamlProxyConfiguration yamlConfig, int port) throws SQLException;
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/AbstractBootstrapInitializer.java
similarity index 61%
copy from shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
copy to shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/AbstractBootstrapInitializer.java
index a57e69c..7db6479 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/Bootstrap.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/AbstractBootstrapInitializer.java
@@ -15,36 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy;
+package org.apache.shardingsphere.proxy.init.impl;
 
-import lombok.AccessLevel;
-import lombok.NoArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.db.protocol.mysql.constant.MySQLServerInfo;
-import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
-import org.apache.shardingsphere.governance.context.schema.GovernanceSchemaContexts;
-import org.apache.shardingsphere.governance.context.transaction.GovernanceTransactionContexts;
 import org.apache.shardingsphere.infra.config.properties.ConfigurationPropertyKey;
 import org.apache.shardingsphere.infra.context.SchemaContext;
 import org.apache.shardingsphere.infra.context.SchemaContexts;
 import org.apache.shardingsphere.infra.context.SchemaContextsBuilder;
-import org.apache.shardingsphere.proxy.arg.BootstrapArguments;
 import org.apache.shardingsphere.proxy.backend.schema.ProxyDataSourceContext;
 import org.apache.shardingsphere.proxy.backend.schema.ProxySchemaContexts;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
-import org.apache.shardingsphere.proxy.config.ProxyConfigurationLoader;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
-import org.apache.shardingsphere.proxy.config.yaml.swapper.YamlProxyConfigurationSwapper;
 import org.apache.shardingsphere.proxy.db.DatabaseServerInfo;
 import org.apache.shardingsphere.proxy.frontend.bootstrap.ShardingSphereProxy;
-import org.apache.shardingsphere.proxy.governance.GovernanceBootstrap;
+import org.apache.shardingsphere.proxy.init.BootstrapInitializer;
 import org.apache.shardingsphere.tracing.opentracing.OpenTracingTracer;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.apache.shardingsphere.transaction.context.impl.StandardTransactionContexts;
 
 import javax.sql.DataSource;
-import java.io.IOException;
 import java.sql.SQLException;
 import java.util.HashMap;
 import java.util.Map;
@@ -52,52 +43,30 @@ import java.util.Map.Entry;
 import java.util.Optional;
 
 /**
- * ShardingSphere-Proxy Bootstrap.
+ * Abstract bootstrap initializer.
  */
-@NoArgsConstructor(access = AccessLevel.PRIVATE)
 @Slf4j
-public final class Bootstrap {
+public abstract class AbstractBootstrapInitializer implements BootstrapInitializer {
     
-    /**
-     * Main entrance.
-     *
-     * @param args startup arguments
-     * @throws IOException IO exception
-     * @throws SQLException SQL exception
-     */
-    public static void main(final String[] args) throws IOException, SQLException {
-        BootstrapArguments bootstrapArgs = new BootstrapArguments(args);
-        int port = bootstrapArgs.getPort();
-        YamlProxyConfiguration yamlConfig = ProxyConfigurationLoader.load(bootstrapArgs.getConfigurationPath());
-        if (null == yamlConfig.getServerConfiguration().getGovernance()) {
-            init(new YamlProxyConfigurationSwapper().swap(yamlConfig), port, null);
-        } else {
-            GovernanceFacade governanceFacade = new GovernanceFacade();
-            init(new GovernanceBootstrap(governanceFacade).init(yamlConfig), port, governanceFacade);
-        }
-    }
-    
-    private static void init(final ProxyConfiguration proxyConfig, final int port, final GovernanceFacade governanceFacade) throws SQLException {
-        SchemaContexts schemaContexts = createSchemaContexts(proxyConfig);
-        TransactionContexts transactionContexts = createTransactionContexts(schemaContexts);
-        if (null != governanceFacade) {
-            schemaContexts = new GovernanceSchemaContexts(schemaContexts, governanceFacade);
-            transactionContexts = new GovernanceTransactionContexts(transactionContexts);
-        }
+    @Override
+    public final void init(final YamlProxyConfiguration yamlConfig, final int port) throws SQLException {
+        ProxyConfiguration proxyConfig = getProxyConfiguration(yamlConfig);
+        SchemaContexts schemaContexts = decorateSchemaContexts(createSchemaContexts(proxyConfig));
+        TransactionContexts transactionContexts = decorateTransactionContexts(createTransactionContexts(schemaContexts));
         ProxySchemaContexts.getInstance().init(schemaContexts, transactionContexts);
         initOpenTracing();
         setDatabaseServerInfo();
         ShardingSphereProxy.getInstance().start(port);
     }
     
-    private static SchemaContexts createSchemaContexts(final ProxyConfiguration proxyConfig) throws SQLException {
+    private SchemaContexts createSchemaContexts(final ProxyConfiguration proxyConfig) throws SQLException {
         ProxyDataSourceContext dataSourceContext = new ProxyDataSourceContext(proxyConfig.getSchemaDataSources());
         SchemaContextsBuilder schemaContextsBuilder = new SchemaContextsBuilder(
                 dataSourceContext.getDatabaseType(), dataSourceContext.getDataSourcesMap(), proxyConfig.getSchemaRules(), proxyConfig.getAuthentication(), proxyConfig.getProps());
         return schemaContextsBuilder.build();
     }
     
-    private static TransactionContexts createTransactionContexts(final SchemaContexts schemaContexts) {
+    private TransactionContexts createTransactionContexts(final SchemaContexts schemaContexts) {
         Map<String, ShardingTransactionManagerEngine> transactionManagerEngines = new HashMap<>(schemaContexts.getSchemaContexts().size(), 1);
         for (Entry<String, SchemaContext> entry : schemaContexts.getSchemaContexts().entrySet()) {
             ShardingTransactionManagerEngine engine = new ShardingTransactionManagerEngine();
@@ -107,13 +76,13 @@ public final class Bootstrap {
         return new StandardTransactionContexts(transactionManagerEngines);
     }
     
-    private static void initOpenTracing() {
+    private void initOpenTracing() {
         if (ProxySchemaContexts.getInstance().getSchemaContexts().getProps().<Boolean>getValue(ConfigurationPropertyKey.PROXY_OPENTRACING_ENABLED)) {
             OpenTracingTracer.init();
         }
     }
     
-    private static void setDatabaseServerInfo() {
+    private void setDatabaseServerInfo() {
         Optional<DataSource> dataSourceSample = ProxySchemaContexts.getInstance().getDataSourceSample();
         if (dataSourceSample.isPresent()) {
             DatabaseServerInfo databaseServerInfo = new DatabaseServerInfo(dataSourceSample.get());
@@ -121,4 +90,10 @@ public final class Bootstrap {
             MySQLServerInfo.setServerVersion(databaseServerInfo.getDatabaseVersion());
         }
     }
+    
+    protected abstract ProxyConfiguration getProxyConfiguration(YamlProxyConfiguration yamlConfig);
+    
+    protected abstract SchemaContexts decorateSchemaContexts(SchemaContexts schemaContexts);
+    
+    protected abstract TransactionContexts decorateTransactionContexts(TransactionContexts transactionContexts);
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrap.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializer.java
similarity index 78%
rename from shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrap.java
rename to shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializer.java
index 2c5bfbd..4158ef2 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrap.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializer.java
@@ -15,22 +15,27 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.governance;
+package org.apache.shardingsphere.proxy.init.impl;
 
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.governance.context.schema.GovernanceSchemaContexts;
+import org.apache.shardingsphere.governance.context.transaction.GovernanceTransactionContexts;
+import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
+import org.apache.shardingsphere.governance.core.yaml.swapper.GovernanceConfigurationYamlSwapper;
 import org.apache.shardingsphere.infra.auth.Authentication;
+import org.apache.shardingsphere.infra.auth.yaml.config.YamlAuthenticationConfiguration;
 import org.apache.shardingsphere.infra.auth.yaml.swapper.AuthenticationYamlSwapper;
-import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
+import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
+import org.apache.shardingsphere.infra.context.SchemaContexts;
 import org.apache.shardingsphere.infra.context.schema.DataSourceParameter;
-import org.apache.shardingsphere.governance.core.yaml.swapper.GovernanceConfigurationYamlSwapper;
-import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
+import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.util.DataSourceParameterConverter;
 import org.apache.shardingsphere.proxy.config.yaml.YamlProxyRuleConfiguration;
 import org.apache.shardingsphere.proxy.config.yaml.YamlProxyServerConfiguration;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
 
 import java.util.Collection;
 import java.util.LinkedHashMap;
@@ -40,20 +45,15 @@ import java.util.Properties;
 import java.util.stream.Collectors;
 
 /**
- * Governance bootstrap.
+ * Governance bootstrap initializer.
  */
 @RequiredArgsConstructor
-public final class GovernanceBootstrap {
+public final class GovernanceBootstrapInitializer extends AbstractBootstrapInitializer {
     
     private final GovernanceFacade governanceFacade;
     
-    /**
-     * Initialize governance.
-     * 
-     * @param yamlConfig YAML proxy configuration
-     * @return proxy configuration
-     */
-    public ProxyConfiguration init(final YamlProxyConfiguration yamlConfig) {
+    @Override
+    protected ProxyConfiguration getProxyConfiguration(final YamlProxyConfiguration yamlConfig) {
         governanceFacade.init(new GovernanceConfigurationYamlSwapper().swapToObject(yamlConfig.getServerConfiguration().getGovernance()), yamlConfig.getRuleConfigurations().keySet());
         initConfigurations(yamlConfig);
         return loadProxyConfiguration();
@@ -65,8 +65,8 @@ public final class GovernanceBootstrap {
         if (isEmptyLocalConfiguration(serverConfig, ruleConfigs)) {
             governanceFacade.onlineInstance();
         } else {
-            governanceFacade.onlineInstance(getDataSourceConfigurationMap(ruleConfigs),
-                    getRuleConfigurations(ruleConfigs), new AuthenticationYamlSwapper().swapToObject(serverConfig.getAuthentication()), serverConfig.getProps());
+            governanceFacade.onlineInstance(
+                    getDataSourceConfigurationMap(ruleConfigs), getRuleConfigurations(ruleConfigs), getAuthentication(serverConfig.getAuthentication()), serverConfig.getProps());
         }
     }
     
@@ -77,7 +77,7 @@ public final class GovernanceBootstrap {
     private Map<String, Map<String, DataSourceConfiguration>> getDataSourceConfigurationMap(final Map<String, YamlProxyRuleConfiguration> ruleConfigs) {
         Map<String, Map<String, DataSourceConfiguration>> result = new LinkedHashMap<>(ruleConfigs.size(), 1);
         for (Entry<String, YamlProxyRuleConfiguration> entry : ruleConfigs.entrySet()) {
-            result.put(entry.getKey(), 
+            result.put(entry.getKey(),
                     DataSourceParameterConverter.getDataSourceConfigurationMap(DataSourceParameterConverter.getDataSourceParameterMapFromYamlConfiguration(entry.getValue().getDataSources())));
         }
         return result;
@@ -88,6 +88,10 @@ public final class GovernanceBootstrap {
         return yamlRuleConfigurations.entrySet().stream().collect(Collectors.toMap(Entry::getKey, entry -> swapperEngine.swapToRuleConfigurations(entry.getValue().getRules())));
     }
     
+    private Authentication getAuthentication(final YamlAuthenticationConfiguration authConfig) {
+        return new AuthenticationYamlSwapper().swapToObject(authConfig);
+    }
+    
     private ProxyConfiguration loadProxyConfiguration() {
         Collection<String> schemaNames = governanceFacade.getConfigCenter().getAllSchemaNames();
         Map<String, Map<String, DataSourceParameter>> schemaDataSources = loadDataSourceParametersMap(schemaNames);
@@ -112,4 +116,14 @@ public final class GovernanceBootstrap {
         }
         return result;
     }
+    
+    @Override
+    protected SchemaContexts decorateSchemaContexts(final SchemaContexts schemaContexts) {
+        return new GovernanceSchemaContexts(schemaContexts, governanceFacade);
+    }
+    
+    @Override
+    protected TransactionContexts decorateTransactionContexts(final TransactionContexts transactionContexts) {
+        return new GovernanceTransactionContexts(transactionContexts);
+    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/StandardBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/StandardBootstrapInitializer.java
new file mode 100644
index 0000000..7b5d65f
--- /dev/null
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/init/impl/StandardBootstrapInitializer.java
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.proxy.init.impl;
+
+import org.apache.shardingsphere.infra.context.SchemaContexts;
+import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
+import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
+import org.apache.shardingsphere.proxy.config.yaml.swapper.YamlProxyConfigurationSwapper;
+import org.apache.shardingsphere.transaction.context.TransactionContexts;
+
+/**
+ * Standard bootstrap initializer.
+ */
+public final class StandardBootstrapInitializer extends AbstractBootstrapInitializer {
+    
+    @Override
+    protected ProxyConfiguration getProxyConfiguration(final YamlProxyConfiguration yamlConfig) {
+        return new YamlProxyConfigurationSwapper().swap(yamlConfig);
+    }
+    
+    @Override
+    protected SchemaContexts decorateSchemaContexts(final SchemaContexts schemaContexts) {
+        return schemaContexts;
+    }
+    
+    @Override
+    protected TransactionContexts decorateTransactionContexts(final TransactionContexts transactionContexts) {
+        return transactionContexts;
+    }
+}
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrapTest.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializerTest.java
similarity index 75%
rename from shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrapTest.java
rename to shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializerTest.java
index b84aeab..eb461d3 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/governance/GovernanceBootstrapTest.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/test/java/org/apache/shardingsphere/proxy/init/impl/GovernanceBootstrapInitializerTest.java
@@ -15,11 +15,11 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.proxy.governance;
+package org.apache.shardingsphere.proxy.init.impl;
 
 import com.zaxxer.hikari.HikariDataSource;
-import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
+import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.ProxyConfigurationLoader;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
@@ -36,30 +36,30 @@ import java.util.Collections;
 import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
-public final class GovernanceBootstrapTest {
+public final class GovernanceBootstrapInitializerTest {
     
     @Mock(answer = Answers.RETURNS_DEEP_STUBS)
     private GovernanceFacade governanceFacade;
     
-    private GovernanceBootstrap governanceBootstrap;
+    private GovernanceBootstrapInitializer initializer;
     
     @Before
     public void setUp() {
-        governanceBootstrap = new GovernanceBootstrap(governanceFacade);
+        initializer = new GovernanceBootstrapInitializer(governanceFacade);
         when(governanceFacade.getConfigCenter().getAllSchemaNames()).thenReturn(Collections.singletonList("db"));
         when(governanceFacade.getConfigCenter().loadDataSourceConfigurations("db")).thenReturn(Collections.singletonMap("db", new DataSourceConfiguration(HikariDataSource.class.getName())));
     }
     
     @Test
-    public void assertInitFromRegistryCenter() throws IOException {
+    public void assertGetProxyConfiguration() throws IOException {
         YamlProxyConfiguration yamlProxyConfig = ProxyConfigurationLoader.load("/conf/reg_center/");
-        assertProxyConfiguration(governanceBootstrap.init(new YamlProxyConfiguration(yamlProxyConfig.getServerConfiguration(), yamlProxyConfig.getRuleConfigurations())));
+        assertProxyConfiguration(initializer.getProxyConfiguration(new YamlProxyConfiguration(yamlProxyConfig.getServerConfiguration(), yamlProxyConfig.getRuleConfigurations())));
     }
     
     @Test
-    public void assertInitFromLocalConfiguration() throws IOException {
+    public void assertGetProxyConfigurationFromLocalConfiguration() throws IOException {
         YamlProxyConfiguration yamlProxyConfig = ProxyConfigurationLoader.load("/conf/local");
-        assertProxyConfiguration(governanceBootstrap.init(new YamlProxyConfiguration(yamlProxyConfig.getServerConfiguration(), yamlProxyConfig.getRuleConfigurations())));
+        assertProxyConfiguration(initializer.getProxyConfiguration(new YamlProxyConfiguration(yamlProxyConfig.getServerConfiguration(), yamlProxyConfig.getRuleConfigurations())));
     }
     
     private void assertProxyConfiguration(final ProxyConfiguration actual) {