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 2022/01/29 11:55:48 UTC

[shardingsphere] branch master updated: Add StandalonePersistRepositoryFactory (#15198)

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 30afd9c  Add StandalonePersistRepositoryFactory (#15198)
30afd9c is described below

commit 30afd9c1e7bdb07a3bdd5c8c716fd90079e0e996
Author: Liang Zhang <te...@163.com>
AuthorDate: Sat Jan 29 19:54:47 2022 +0800

    Add StandalonePersistRepositoryFactory (#15198)
---
 .../StandaloneContextManagerBuilder.java           | 35 +++-------------
 .../standalone/StandalonePersistRepository.java    |  3 +-
 .../StandalonePersistRepositoryFactory.java        | 47 ++++++++++++++++++++++
 .../repository/standalone/file/FileRepository.java |  5 +++
 4 files changed, 60 insertions(+), 30 deletions(-)

diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
index 32d288b..a94dc5a 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
@@ -19,7 +19,6 @@ package org.apache.shardingsphere.mode.manager.standalone;
 
 import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
-import org.apache.shardingsphere.infra.config.mode.PersistRepositoryConfiguration;
 import org.apache.shardingsphere.infra.config.schema.SchemaConfiguration;
 import org.apache.shardingsphere.infra.config.schema.impl.DataSourceProvidedSchemaConfiguration;
 import org.apache.shardingsphere.infra.datasource.pool.creator.DataSourcePoolCreator;
@@ -28,9 +27,6 @@ import org.apache.shardingsphere.infra.datasource.props.DataSourceProperties;
 import org.apache.shardingsphere.infra.datasource.props.DataSourcePropertiesCreator;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.definition.InstanceType;
-import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
-import org.apache.shardingsphere.infra.metadata.schema.loader.SchemaLoader;
-import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.InstanceAwareRule;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilder;
@@ -39,10 +35,7 @@ import org.apache.shardingsphere.mode.manager.standalone.workerid.generator.Stan
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.MetaDataContextsBuilder;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
-import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepository;
-import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepositoryConfiguration;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepositoryFactory;
 import org.apache.shardingsphere.transaction.context.TransactionContexts;
 import org.apache.shardingsphere.transaction.context.TransactionContextsBuilder;
 
@@ -61,16 +54,9 @@ import java.util.stream.Collectors;
 @Slf4j
 public final class StandaloneContextManagerBuilder implements ContextManagerBuilder {
     
-    static {
-        ShardingSphereServiceLoader.register(StandalonePersistRepository.class);
-    }
-    
     @Override
     public ContextManager build(final ContextManagerBuilderParameter parameter) throws SQLException {
-        PersistRepositoryConfiguration repositoryConfig = null == parameter.getModeConfig().getRepository() ? new StandalonePersistRepositoryConfiguration("File", new Properties())
-                : parameter.getModeConfig().getRepository();
-        StandalonePersistRepository repository = TypedSPIRegistry.getRegisteredService(StandalonePersistRepository.class, repositoryConfig.getType(), repositoryConfig.getProps());
-        MetaDataPersistService metaDataPersistService = new MetaDataPersistService(repository);
+        MetaDataPersistService metaDataPersistService = new MetaDataPersistService(StandalonePersistRepositoryFactory.newInstance(parameter.getModeConfig().getRepository()));
         persistConfigurations(metaDataPersistService, parameter.getSchemaConfigs(), parameter.getGlobalRuleConfigs(), parameter.getProps(), parameter.getModeConfig().isOverwrite());
         Collection<String> schemaNames = parameter.getInstanceDefinition().getInstanceType() == InstanceType.JDBC ? parameter.getSchemaConfigs().keySet()
                 : metaDataPersistService.getSchemaMetaDataService().loadAllNames();
@@ -86,7 +72,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
         ContextManager result = new ContextManager();
         result.init(metaDataContexts, transactionContexts, new InstanceContext(metaDataPersistService.getComputeNodePersistService().loadComputeNodeInstance(parameter.getInstanceDefinition()), 
                 new StandaloneWorkerIdGenerator(), getType()));
-        buildSpecialRules(result);
+        setInstanceContext(result);
         return result;
     }
     
@@ -181,21 +167,12 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
             each -> each, each -> metaDataPersistService.getSchemaRuleService().load(each), (oldValue, currentValue) -> oldValue, LinkedHashMap::new));
     }
     
-    private void buildSpecialRules(final ContextManager contextManager) {
-        contextManager.getMetaDataContexts().getMetaDataMap().forEach((key, value)
-            -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
+    private void setInstanceContext(final ContextManager contextManager) {
+        contextManager.getMetaDataContexts().getMetaDataMap()
+            .forEach((key, value) -> value.getRuleMetaData().getRules().stream().filter(each -> each instanceof InstanceAwareRule)
             .forEach(each -> ((InstanceAwareRule) each).setInstanceContext(contextManager.getInstanceContext())));
     }
     
-    private Map<String, ShardingSphereSchema> getShardingSphereSchemas(final Map<String, ? extends SchemaConfiguration> schemaConfigs, final Map<String, Collection<ShardingSphereRule>> rules,
-                                                                       final Properties props) throws SQLException {
-        Map<String, ShardingSphereSchema> result = new LinkedHashMap<>(schemaConfigs.size(), 1);
-        for (String each : schemaConfigs.keySet()) {
-            result.put(each, SchemaLoader.load(schemaConfigs.get(each).getDataSources(), rules.get(each), props));
-        }
-        return result;
-    }
-    
     @Override
     public String getType() {
         return "Standalone";
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repository/standalone/StandalonePersistRepository.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repos [...]
index 890351d..82001b5 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repository/standalone/StandalonePersistRepository.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repository/standalone/StandalonePersistRepository.java
@@ -18,9 +18,10 @@
 package org.apache.shardingsphere.mode.repository.standalone;
 
 import org.apache.shardingsphere.mode.persist.PersistRepository;
+import org.apache.shardingsphere.spi.required.RequiredSPI;
 
 /**
  * Standalone persist repository.
  */
-public interface StandalonePersistRepository extends PersistRepository {
+public interface StandalonePersistRepository extends PersistRepository, RequiredSPI {
 }
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repository/standalone/StandalonePersistRepositoryFactory.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mod [...]
new file mode 100644
index 0000000..bf10e35
--- /dev/null
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-api/src/main/java/org/apache/shardingsphere/mode/repository/standalone/StandalonePersistRepositoryFactory.java
@@ -0,0 +1,47 @@
+/*
+ * 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.mode.repository.standalone;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.config.mode.PersistRepositoryConfiguration;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.required.RequiredSPIRegistry;
+import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
+
+/**
+ * Standalone persist repository factory.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class StandalonePersistRepositoryFactory {
+    
+    static {
+        ShardingSphereServiceLoader.register(StandalonePersistRepository.class);
+    }
+    
+    /**
+     * Create new instance of standalone persist repository.
+     * 
+     * @param repositoryConfig persist repository configuration
+     * @return new instance of standalone persist repository
+     */
+    public static StandalonePersistRepository newInstance(final PersistRepositoryConfiguration repositoryConfig) {
+        return null == repositoryConfig ? RequiredSPIRegistry.getRegisteredService(StandalonePersistRepository.class)
+                : TypedSPIRegistry.getRegisteredService(StandalonePersistRepository.class, repositoryConfig.getType(), repositoryConfig.getProps());
+    }
+}
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-provider/shardingsphere-standalone-mode-repository-file/src/main/java/org/apache/shardingsphere/mode/repository/standalone/file/FileRepository.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-provider/s [...]
index 9cad283..8945bb6 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-provider/shardingsphere-standalone-mode-repository-file/src/main/java/org/apache/shardingsphere/mode/repository/standalone/file/FileRepository.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-standalone-mode/shardingsphere-standalone-mode-repository/shardingsphere-standalone-mode-repository-provider/shardingsphere-standalone-mode-repository-file/src/main/java/org/apache/shardingsphere/mode/repository/standalone/file/FileRepository.java
@@ -107,6 +107,11 @@ public final class FileRepository implements StandalonePersistRepository {
     }
     
     @Override
+    public boolean isDefault() {
+        return true;
+    }
+    
+    @Override
     public String getType() {
         return "File";
     }