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 2021/08/22 04:39:25 UTC

[shardingsphere] branch master updated: Merge ShardingSphereMode and ContextManager (#11940)

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 93e3e8e  Merge ShardingSphereMode and ContextManager (#11940)
93e3e8e is described below

commit 93e3e8ee21dca92ca5c15120c1770fca771995da
Author: Liang Zhang <te...@163.com>
AuthorDate: Sun Aug 22 12:38:52 2021 +0800

    Merge ShardingSphereMode and ContextManager (#11940)
---
 .../context/ClusterContextManagerBuilder.java      | 27 +++++++++--
 .../governance/core/mode/ClusterMode.java          | 33 -------------
 .../governance/core/mode/ClusterModeBuilder.java   | 54 ----------------------
 ...e.shardingsphere.infra.mode.builder.ModeBuilder | 18 --------
 .../infra/mode/ShardingSphereMode.java             | 24 ----------
 .../infra/mode/builder/ModeBuilder.java            | 36 ---------------
 .../infra/mode/builder/ModeBuilderEngine.java      | 46 ------------------
 .../infra/mode/impl/memory/MemoryMode.java         | 26 -----------
 .../infra/mode/impl/memory/MemoryModeBuilder.java  | 38 ---------------
 .../infra/mode/impl/standalone/StandaloneMode.java | 32 -------------
 .../impl/standalone/StandaloneModeBuilder.java     | 48 -------------------
 .../standalone/StandalonePersistRepository.java    |  2 +-
 .../StandalonePersistRepositoryConfiguration.java  |  2 +-
 .../persist/repository/local/LocalRepository.java  |  2 +-
 ...ePersistRepositoryConfigurationYamlSwapper.java |  2 +-
 ...e.shardingsphere.infra.mode.builder.ModeBuilder | 19 --------
 ...ra.mode.standalone.StandalonePersistRepository} |  0
 .../context/manager/ContextManagerBuilder.java     | 10 ++--
 .../manager/impl/MemoryContextManagerBuilder.java  |  8 ++--
 .../impl/StandaloneContextManagerBuilder.java      | 21 +++++++--
 .../core/datasource/ShardingSphereDataSource.java  | 10 +---
 ...epositoryConfigurationBeanDefinitionParser.java |  2 +-
 .../org/apache/shardingsphere/proxy/Bootstrap.java | 11 +----
 .../proxy/initializer/BootstrapInitializer.java    |  6 +--
 .../scaling/core/config/ServerConfiguration.java   |  1 -
 25 files changed, 55 insertions(+), 423 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/ClusterContextManagerBuilder.java b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/ClusterContextManagerBuilder.java
index 30baf67..b710e8c 100644
--- a/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/ClusterContextManagerBuilder.java
+++ b/shardingsphere-governance/shardingsphere-governance-context/src/main/java/org/apache/shardingsphere/governance/context/ClusterContextManagerBuilder.java
@@ -17,8 +17,10 @@
 
 package org.apache.shardingsphere.governance.context;
 
-import org.apache.shardingsphere.governance.core.mode.ClusterMode;
+import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.governance.core.registry.RegistryCenter;
+import org.apache.shardingsphere.governance.repository.api.config.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.governance.repository.spi.ClusterPersistRepository;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration;
 import org.apache.shardingsphere.infra.config.datasource.DataSourceConverter;
@@ -28,7 +30,10 @@ import org.apache.shardingsphere.infra.context.manager.ContextManagerBuilder;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContextsBuilder;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
+import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
 import org.apache.shardingsphere.infra.persist.PersistService;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 
@@ -45,14 +50,19 @@ import java.util.stream.Collectors;
 /**
  * Cluster context manager builder.
  */
-public final class ClusterContextManagerBuilder implements ContextManagerBuilder<ClusterMode> {
+public final class ClusterContextManagerBuilder implements ContextManagerBuilder {
+    
+    static {
+        ShardingSphereServiceLoader.register(ClusterPersistRepository.class);
+    }
     
     @Override
-    public ContextManager build(final ClusterMode mode, final Map<String, Map<String, DataSource>> dataSourcesMap,
+    public ContextManager build(final ModeConfiguration modeConfig, final Map<String, Map<String, DataSource>> dataSourcesMap,
                                 final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs,
                                 final Properties props, final boolean isOverwrite) throws SQLException {
-        PersistService persistService = new PersistService(mode.getRepository());
-        RegistryCenter registryCenter = new RegistryCenter(mode.getRepository());
+        ClusterPersistRepository repository = createClusterPersistRepository((ClusterPersistRepositoryConfiguration) modeConfig.getRepository());
+        PersistService persistService = new PersistService(repository);
+        RegistryCenter registryCenter = new RegistryCenter(repository);
         persistConfigurations(persistService, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, props, isOverwrite);
         // TODO Here may be some problems to load all schemaNames for JDBC
         Collection<String> schemaNames = persistService.getSchemaMetaDataService().loadAllNames();
@@ -71,6 +81,13 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
         return result;
     }
     
+    private ClusterPersistRepository createClusterPersistRepository(final ClusterPersistRepositoryConfiguration config) {
+        Preconditions.checkNotNull(config, "Cluster persist repository configuration cannot be null.");
+        ClusterPersistRepository result = TypedSPIRegistry.getRegisteredService(ClusterPersistRepository.class, config.getType(), config.getProps());
+        result.init(config);
+        return result;
+    }
+    
     private void persistConfigurations(final PersistService persistService, final Map<String, Map<String, DataSource>> dataSourcesMap,
                                        final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs,
                                        final Properties props, final boolean overwrite) {
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterMode.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterMode.java
deleted file mode 100644
index 5c4ccb7..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterMode.java
+++ /dev/null
@@ -1,33 +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.governance.core.mode;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.governance.repository.spi.ClusterPersistRepository;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-
-/**
- * Cluster mode.
- */
-@RequiredArgsConstructor
-@Getter
-public final class ClusterMode implements ShardingSphereMode {
-    
-    private final ClusterPersistRepository repository;
-}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterModeBuilder.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterModeBuilder.java
deleted file mode 100644
index ef11434..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/mode/ClusterModeBuilder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.governance.core.mode;
-
-import com.google.common.base.Preconditions;
-import org.apache.shardingsphere.governance.repository.api.config.ClusterPersistRepositoryConfiguration;
-import org.apache.shardingsphere.governance.repository.spi.ClusterPersistRepository;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.builder.ModeBuilder;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
-
-/**
- * Cluster mode builder.
- */
-public final class ClusterModeBuilder implements ModeBuilder {
-    
-    static {
-        ShardingSphereServiceLoader.register(ClusterPersistRepository.class);
-    }
-    
-    @Override
-    public ShardingSphereMode build(final ModeConfiguration config) {
-        return new ClusterMode(createClusterPersistRepository((ClusterPersistRepositoryConfiguration) config.getRepository()));
-    }
-    
-    private ClusterPersistRepository createClusterPersistRepository(final ClusterPersistRepositoryConfiguration config) {
-        Preconditions.checkNotNull(config, "Cluster persist repository configuration cannot be null.");
-        ClusterPersistRepository result = TypedSPIRegistry.getRegisteredService(ClusterPersistRepository.class, config.getType(), config.getProps());
-        result.init(config);
-        return result;
-    }
-    
-    @Override
-    public String getType() {
-        return "Cluster";
-    }
-}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder b/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder
deleted file mode 100644
index 1826538..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder
+++ /dev/null
@@ -1,18 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.shardingsphere.governance.core.mode.ClusterModeBuilder
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/ShardingSphereMode.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/ShardingSphereMode.java
deleted file mode 100644
index 277083f..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/ShardingSphereMode.java
+++ /dev/null
@@ -1,24 +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.infra.mode;
-
-/**
- * ShardingSphere mode.
- */
-public interface ShardingSphereMode {
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilder.java
deleted file mode 100644
index f8c5a1a..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilder.java
+++ /dev/null
@@ -1,36 +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.infra.mode.builder;
-
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-import org.apache.shardingsphere.infra.spi.typed.TypedSPI;
-
-/**
- * Mode builder.
- */
-public interface ModeBuilder extends TypedSPI {
-    
-    /**
-     * Build mode.
-     * 
-     * @param config mode configuration
-     * @return built mode
-     */
-    ShardingSphereMode build(ModeConfiguration config);
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilderEngine.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilderEngine.java
deleted file mode 100644
index d143358..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/builder/ModeBuilderEngine.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.mode.builder;
-
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-import org.apache.shardingsphere.infra.mode.impl.memory.MemoryMode;
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
-
-import java.util.Properties;
-
-/**
- * Mode builder engine.
- */
-public final class ModeBuilderEngine {
-    
-    static {
-        ShardingSphereServiceLoader.register(ModeBuilder.class);
-    }
-    
-    /**
-     * Build mode.
-     * 
-     * @param config mode configuration
-     * @return built mode
-     */
-    public static ShardingSphereMode build(final ModeConfiguration config) {
-        return null == config ? new MemoryMode() : TypedSPIRegistry.getRegisteredService(ModeBuilder.class, config.getType(), new Properties()).build(config);
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryMode.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryMode.java
deleted file mode 100644
index 868df9a..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryMode.java
+++ /dev/null
@@ -1,26 +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.infra.mode.impl.memory;
-
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-
-/**
- * Memory mode.
- */
-public final class MemoryMode implements ShardingSphereMode {
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryModeBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryModeBuilder.java
deleted file mode 100644
index df2fae1..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/memory/MemoryModeBuilder.java
+++ /dev/null
@@ -1,38 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.shardingsphere.infra.mode.impl.memory;
-
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.builder.ModeBuilder;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-
-/**
- * Memory mode builder.
- */
-public final class MemoryModeBuilder implements ModeBuilder {
-    
-    @Override
-    public ShardingSphereMode build(final ModeConfiguration config) {
-        return new MemoryMode();
-    }
-    
-    @Override
-    public String getType() {
-        return "Memory";
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneMode.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneMode.java
deleted file mode 100644
index eff11bc..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneMode.java
+++ /dev/null
@@ -1,32 +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.infra.mode.impl.standalone;
-
-import lombok.Getter;
-import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-
-/**
- * Standalone mode.
- */
-@RequiredArgsConstructor
-@Getter
-public final class StandaloneMode implements ShardingSphereMode {
-    
-    private final StandalonePersistRepository repository;
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneModeBuilder.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneModeBuilder.java
deleted file mode 100644
index 30c4ba5..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandaloneModeBuilder.java
+++ /dev/null
@@ -1,48 +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.infra.mode.impl.standalone;
-
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.builder.ModeBuilder;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-import org.apache.shardingsphere.infra.mode.config.PersistRepositoryConfiguration;
-import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
-
-import java.util.Properties;
-
-/**
- * Standalone mode builder.
- */
-public final class StandaloneModeBuilder implements ModeBuilder {
-    
-    static {
-        ShardingSphereServiceLoader.register(StandalonePersistRepository.class);
-    }
-    
-    @Override
-    public ShardingSphereMode build(final ModeConfiguration config) {
-        PersistRepositoryConfiguration repositoryConfig = null == config.getRepository() ? new StandalonePersistRepositoryConfiguration("Local", new Properties()) : config.getRepository();
-        return new StandaloneMode(TypedSPIRegistry.getRegisteredService(StandalonePersistRepository.class, repositoryConfig.getType(), repositoryConfig.getProps()));
-    }
-    
-    @Override
-    public String getType() {
-        return "Standalone";
-    }
-}
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepository.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepository.java
similarity index 94%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepository.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepository.java
index 6dd2a02..47b2e83 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepository.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepository.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.mode.impl.standalone;
+package org.apache.shardingsphere.infra.mode.standalone;
 
 import org.apache.shardingsphere.infra.mode.repository.PersistRepository;
 
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepositoryConfiguration.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepositoryConfiguration.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepositoryConfiguration.java
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepositoryConfiguration.java
index f59e329..aae1bf1 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/impl/standalone/StandalonePersistRepositoryConfiguration.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/mode/standalone/StandalonePersistRepositoryConfiguration.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.mode.impl.standalone;
+package org.apache.shardingsphere.infra.mode.standalone;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/persist/repository/local/LocalRepository.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/persist/repository/local/LocalRepository.java
index 2338630..ea15d4c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/persist/repository/local/LocalRepository.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/persist/repository/local/LocalRepository.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.infra.persist.repository.local;
 import com.google.common.base.Joiner;
 import com.google.common.base.Strings;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.infra.mode.impl.standalone.StandalonePersistRepository;
+import org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepository;
 
 import java.io.BufferedWriter;
 import java.io.File;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/mode/StandalonePersistRepositoryConfigurationYamlSwapper.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/mode/StandalonePersistRepositoryConfigurationYamlSwapper.java
index 427227e..5658466 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/mode/StandalonePersistRepositoryConfigurationYamlSwapper.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/mode/StandalonePersistRepositoryConfigurationYamlSwapper.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.infra.yaml.config.swapper.mode;
 
-import org.apache.shardingsphere.infra.mode.impl.standalone.StandalonePersistRepositoryConfiguration;
+import org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepositoryConfiguration;
 import org.apache.shardingsphere.infra.yaml.config.pojo.mode.YamlPersistRepositoryConfiguration;
 
 /**
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder
deleted file mode 100644
index 63eaeef..0000000
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.builder.ModeBuilder
+++ /dev/null
@@ -1,19 +0,0 @@
-#
-# Licensed to the Apache Software Foundation (ASF) under one or more
-# contributor license agreements.  See the NOTICE file distributed with
-# this work for additional information regarding copyright ownership.
-# The ASF licenses this file to You under the Apache License, Version 2.0
-# (the "License"); you may not use this file except in compliance with
-# the License.  You may obtain a copy of the License at
-#
-#     http://www.apache.org/licenses/LICENSE-2.0
-#
-# Unless required by applicable law or agreed to in writing, software
-# distributed under the License is distributed on an "AS IS" BASIS,
-# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-# See the License for the specific language governing permissions and
-# limitations under the License.
-#
-
-org.apache.shardingsphere.infra.mode.impl.memory.MemoryModeBuilder
-org.apache.shardingsphere.infra.mode.impl.standalone.StandaloneModeBuilder
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.impl.standalone.StandalonePersistRepository b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepository
similarity index 100%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.impl.standalone.StandalonePersistRepository
rename to shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepository
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/ContextManagerBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/ContextManagerBuilder.java
index 74e5b14..ef20496 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/ContextManagerBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/ContextManagerBuilder.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.infra.context.manager;
 
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
+import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPI;
 
 import javax.sql.DataSource;
@@ -29,15 +29,13 @@ import java.util.Properties;
 
 /**
  * Context manager builder.
- * 
- * @param <T> type of mode
  */
-public interface ContextManagerBuilder<T extends ShardingSphereMode> extends TypedSPI {
+public interface ContextManagerBuilder extends TypedSPI {
     
     /**
      * Build context manager.
      * 
-     * @param mode ShardingSphere mode
+     * @param modeConfig mode configuration
      * @param dataSourcesMap data sources map
      * @param schemaRuleConfigs schema rule configurations
      * @param globalRuleConfigs global rule configurations
@@ -46,6 +44,6 @@ public interface ContextManagerBuilder<T extends ShardingSphereMode> extends Typ
      * @return context manager
      * @throws SQLException SQL exception
      */
-    ContextManager build(T mode, Map<String, Map<String, DataSource>> dataSourcesMap,
+    ContextManager build(ModeConfiguration modeConfig, Map<String, Map<String, DataSource>> dataSourcesMap,
                          Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, Collection<RuleConfiguration> globalRuleConfigs, Properties props, boolean isOverwrite) throws SQLException;
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/MemoryContextManagerBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/MemoryContextManagerBuilder.java
index 042d835..9688641 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/MemoryContextManagerBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/MemoryContextManagerBuilder.java
@@ -24,7 +24,7 @@ import org.apache.shardingsphere.infra.context.manager.ContextManagerBuilder;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContextsBuilder;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
-import org.apache.shardingsphere.infra.mode.impl.memory.MemoryMode;
+import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 
@@ -38,11 +38,11 @@ import java.util.Properties;
 /**
  * Memory context manager builder.
  */
-public final class MemoryContextManagerBuilder implements ContextManagerBuilder<MemoryMode> {
+public final class MemoryContextManagerBuilder implements ContextManagerBuilder {
     
     @Override
-    public ContextManager build(final MemoryMode mode, final Map<String, Map<String, DataSource>> dataSourcesMap, 
-                                final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs, 
+    public ContextManager build(final ModeConfiguration modeConfig, final Map<String, Map<String, DataSource>> dataSourcesMap,
+                                final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs,
                                 final Properties props, final boolean isOverwrite) throws SQLException {
         MetaDataContexts metaDataContexts = new MetaDataContextsBuilder(dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, props).build(null);
         TransactionContexts transactionContexts = createTransactionContexts(metaDataContexts);
diff --git a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/StandaloneContextManagerBuilder.java b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/StandaloneContextManagerBuilder.java
index 6833834..8e0336e 100644
--- a/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/StandaloneContextManagerBuilder.java
+++ b/shardingsphere-infra/shardingsphere-infra-context/src/main/java/org/apache/shardingsphere/infra/context/manager/impl/StandaloneContextManagerBuilder.java
@@ -26,8 +26,13 @@ import org.apache.shardingsphere.infra.context.manager.ContextManagerBuilder;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContextsBuilder;
 import org.apache.shardingsphere.infra.metadata.resource.ShardingSphereResource;
-import org.apache.shardingsphere.infra.mode.impl.standalone.StandaloneMode;
+import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
+import org.apache.shardingsphere.infra.mode.config.PersistRepositoryConfiguration;
+import org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepository;
+import org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepositoryConfiguration;
 import org.apache.shardingsphere.infra.persist.PersistService;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 import org.apache.shardingsphere.transaction.ShardingTransactionManagerEngine;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 
@@ -44,13 +49,19 @@ import java.util.stream.Collectors;
 /**
  * Standalone context manager builder.
  */
-public final class StandaloneContextManagerBuilder implements ContextManagerBuilder<StandaloneMode> {
+public final class StandaloneContextManagerBuilder implements ContextManagerBuilder {
+    
+    static {
+        ShardingSphereServiceLoader.register(StandalonePersistRepository.class);
+    }
     
     @Override
-    public ContextManager build(final StandaloneMode mode, final Map<String, Map<String, DataSource>> dataSourcesMap, 
-                                final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs, 
+    public ContextManager build(final ModeConfiguration modeConfig, final Map<String, Map<String, DataSource>> dataSourcesMap,
+                                final Map<String, Collection<RuleConfiguration>> schemaRuleConfigs, final Collection<RuleConfiguration> globalRuleConfigs,
                                 final Properties props, final boolean isOverwrite) throws SQLException {
-        PersistService persistService = new PersistService(mode.getRepository());
+        PersistRepositoryConfiguration repositoryConfig = null == modeConfig.getRepository() ? new StandalonePersistRepositoryConfiguration("Local", new Properties()) : modeConfig.getRepository();
+        StandalonePersistRepository repository = TypedSPIRegistry.getRegisteredService(StandalonePersistRepository.class, repositoryConfig.getType(), repositoryConfig.getProps());
+        PersistService persistService = new PersistService(repository);
         persistConfigurations(persistService, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, props, isOverwrite);
         Collection<String> schemaNames = persistService.getSchemaMetaDataService().loadAllNames();
         MetaDataContexts metaDataContexts;
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
index c78ed69..ec9c3ed 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/jdbc/core/datasource/ShardingSphereDataSource.java
@@ -25,8 +25,6 @@ import org.apache.shardingsphere.infra.config.scope.GlobalRuleConfiguration;
 import org.apache.shardingsphere.infra.config.scope.SchemaRuleConfiguration;
 import org.apache.shardingsphere.infra.context.manager.ContextManager;
 import org.apache.shardingsphere.infra.context.manager.ContextManagerBuilder;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.builder.ModeBuilderEngine;
 import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
@@ -67,26 +65,22 @@ public final class ShardingSphereDataSource extends AbstractUnsupportedOperation
         contextManager = createContextManager(schemaName, modeConfig, dataSourceMap, ruleConfigs, props);
     }
     
-    @SuppressWarnings({"rawtypes", "unchecked"})
     private ContextManager createContextManager(final String schemaName, final ModeConfiguration modeConfig) throws SQLException {
-        ShardingSphereMode mode = ModeBuilderEngine.build(modeConfig);
         Map<String, Map<String, DataSource>> dataSourcesMap = Collections.singletonMap(schemaName, new HashMap<>());
         Map<String, Collection<RuleConfiguration>> schemaRuleConfigs = Collections.singletonMap(schemaName, Collections.emptyList());
         Collection<RuleConfiguration> globalRuleConfigs = Collections.emptyList();
         ContextManagerBuilder builder = TypedSPIRegistry.getRegisteredService(ContextManagerBuilder.class, null == modeConfig ? "Memory" : modeConfig.getType(), new Properties());
-        return builder.build(mode, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, new Properties(), false);
+        return builder.build(modeConfig, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, new Properties(), false);
     }
     
-    @SuppressWarnings({"rawtypes", "unchecked"})
     private ContextManager createContextManager(final String schemaName, final ModeConfiguration modeConfig, final Map<String, DataSource> dataSourceMap,
                                                 final Collection<RuleConfiguration> ruleConfigs, final Properties props) throws SQLException {
-        ShardingSphereMode mode = ModeBuilderEngine.build(modeConfig);
         Map<String, Map<String, DataSource>> dataSourcesMap = Collections.singletonMap(schemaName, dataSourceMap);
         Map<String, Collection<RuleConfiguration>> schemaRuleConfigs = Collections.singletonMap(
                 schemaName, ruleConfigs.stream().filter(each -> each instanceof SchemaRuleConfiguration).collect(Collectors.toList()));
         Collection<RuleConfiguration> globalRuleConfigs = ruleConfigs.stream().filter(each -> each instanceof GlobalRuleConfiguration).collect(Collectors.toList());
         ContextManagerBuilder builder = TypedSPIRegistry.getRegisteredService(ContextManagerBuilder.class, null == modeConfig ? "Memory" : modeConfig.getType(), new Properties());
-        return builder.build(mode, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, props, null == modeConfig || modeConfig.isOverwrite());
+        return builder.build(modeConfig, dataSourcesMap, schemaRuleConfigs, globalRuleConfigs, props, null == modeConfig || modeConfig.isOverwrite());
     }
     
     @Override
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/main/java/org/apache/shardingsphere/spring/namespace/parser/StandaloneModeRepositoryConfigurationBeanDefinitionParser.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/main/java/org/apache/shardingsphere/spring/namespace/parser/StandaloneModeRepositoryConfigurationBeanDefi [...]
index 00db9a1..fdade38 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/main/java/org/apache/shardingsphere/spring/namespace/parser/StandaloneModeRepositoryConfigurationBeanDefinitionParser.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-core-spring/shardingsphere-jdbc-core-spring-namespace/src/main/java/org/apache/shardingsphere/spring/namespace/parser/StandaloneModeRepositoryConfigurationBeanDefinitionParser.java
@@ -17,7 +17,7 @@
 
 package org.apache.shardingsphere.spring.namespace.parser;
 
-import org.apache.shardingsphere.infra.mode.impl.standalone.StandalonePersistRepositoryConfiguration;
+import org.apache.shardingsphere.infra.mode.standalone.StandalonePersistRepositoryConfiguration;
 import org.apache.shardingsphere.spring.namespace.tag.StandaloneModeRepositoryBeanDefinitionTag;
 import org.springframework.beans.factory.support.AbstractBeanDefinition;
 import org.springframework.beans.factory.support.BeanDefinitionBuilder;
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 5ffcd72..6fb0309 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,10 +19,6 @@ package org.apache.shardingsphere.proxy;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
-import org.apache.shardingsphere.infra.mode.builder.ModeBuilderEngine;
-import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
-import org.apache.shardingsphere.infra.yaml.config.swapper.mode.ModeConfigurationYamlSwapper;
 import org.apache.shardingsphere.proxy.arguments.BootstrapArguments;
 import org.apache.shardingsphere.proxy.config.ProxyConfigurationLoader;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
@@ -48,12 +44,7 @@ public final class Bootstrap {
     public static void main(final String[] args) throws IOException, SQLException {
         BootstrapArguments bootstrapArgs = new BootstrapArguments(args);
         YamlProxyConfiguration yamlConfig = ProxyConfigurationLoader.load(bootstrapArgs.getConfigurationPath());
-        ShardingSphereMode mode = ModeBuilderEngine.build(getModeConfiguration(yamlConfig));
-        new BootstrapInitializer(mode).init(yamlConfig);
+        new BootstrapInitializer().init(yamlConfig);
         new ShardingSphereProxy().start(bootstrapArgs.getPort());
     }
-    
-    private static ModeConfiguration getModeConfiguration(final YamlProxyConfiguration yamlConfig) {
-        return null == yamlConfig.getServerConfiguration().getMode() ? null : new ModeConfigurationYamlSwapper().swapToObject(yamlConfig.getServerConfiguration().getMode());
-    }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
index 90cb826..9b143bc 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/BootstrapInitializer.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration
 import org.apache.shardingsphere.infra.config.datasource.DataSourceParameter;
 import org.apache.shardingsphere.infra.context.manager.ContextManager;
 import org.apache.shardingsphere.infra.context.manager.ContextManagerBuilder;
-import org.apache.shardingsphere.infra.mode.ShardingSphereMode;
 import org.apache.shardingsphere.infra.mode.config.ModeConfiguration;
 import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
@@ -59,21 +58,18 @@ public final class BootstrapInitializer {
         ShardingSphereServiceLoader.register(ContextManagerBuilder.class);
     }
     
-    private final ShardingSphereMode mode;
-    
     /**
      * Initialize.
      *
      * @param yamlConfig YAML proxy configuration
      * @throws SQLException SQL exception
      */
-    @SuppressWarnings("unchecked")
     public void init(final YamlProxyConfiguration yamlConfig) throws SQLException {
         ProxyConfiguration proxyConfig = new YamlProxyConfigurationSwapper().swap(yamlConfig);
         ModeConfiguration modeConfig = null == yamlConfig.getServerConfiguration().getMode()
                 ? new ModeConfiguration("Memory", null, true) : new ModeConfigurationYamlSwapper().swapToObject(yamlConfig.getServerConfiguration().getMode());
         ContextManager contextManager = TypedSPIRegistry.getRegisteredService(ContextManagerBuilder.class, modeConfig.getType(), new Properties()).build(
-                mode, getDataSourcesMap(proxyConfig.getSchemaDataSources()), proxyConfig.getSchemaRules(), proxyConfig.getGlobalRules(), proxyConfig.getProps(), modeConfig.isOverwrite());
+                modeConfig, getDataSourcesMap(proxyConfig.getSchemaDataSources()), proxyConfig.getSchemaRules(), proxyConfig.getGlobalRules(), proxyConfig.getProps(), modeConfig.isOverwrite());
         ProxyContext.getInstance().init(contextManager);
         setDatabaseServerInfo();
         initScaling(yamlConfig, modeConfig);
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/config/ServerConfiguration.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/config/ServerConfiguration.java
index 4eb265e..3d319dd 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/config/ServerConfiguration.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/config/ServerConfiguration.java
@@ -34,6 +34,5 @@ public final class ServerConfiguration {
     
     private int workerThread = 30;
     
-    // TODO try to use ShardingSphereMode to instead of ModeConfiguration
     private ModeConfiguration modeConfiguration;
 }