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 2022/01/10 03:58:03 UTC

[shardingsphere] branch master updated: Add SingletonSPI and SingletonSPIRegistry; RequiredSPIRegistry.getRegisteredService support SingletonSPI; Refactor part of existing SPI loading (#14640)

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

zhangliang 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 dc02181  Add SingletonSPI and SingletonSPIRegistry; RequiredSPIRegistry.getRegisteredService support SingletonSPI; Refactor part of existing SPI loading (#14640)
dc02181 is described below

commit dc0218196d69296ff04537d72b66e8157255e35c
Author: Hongsheng Zhong <sa...@126.com>
AuthorDate: Mon Jan 10 11:57:18 2022 +0800

    Add SingletonSPI and SingletonSPIRegistry; RequiredSPIRegistry.getRegisteredService support SingletonSPI; Refactor part of existing SPI loading (#14640)
    
    * Add SingletonSPI and SingletonSPIRegistry
    
    * Refactor ShardingSphereServiceLoader.getSingletonServiceInstances to SingletonSPIRegistry invocation
    
    * Refactor part of TypedSPIRegistry.getRegisteredService to SingletonSPIRegistry invocation
    
    * RequiredSPIRegistry.getRegisteredService support SingletonSPI
    
    * Revert server.yaml
---
 .../builder/loader/TableMetaDataLoaderEngine.java  | 16 ++----
 .../builder/spi/DialectTableMetaDataLoader.java    |  3 +-
 .../dialect/MySQLTableMetaDataLoaderTest.java      | 16 +++---
 .../dialect/OracleTableMetaDataLoaderTest.java     | 16 +++---
 .../dialect/SQLServerTableMetaDataLoaderTest.java  | 16 +++---
 .../shardingsphere/driver/state/DriverState.java   |  3 +-
 .../driver/state/DriverStateContext.java           | 12 +----
 .../prop/DataSourcePropertiesSetter.java           |  3 +-
 .../impl/DataSourcePropertiesSetterHolder.java     | 10 ++--
 .../creator/PipelineDataSourceCreator.java         |  3 +-
 .../creator/PipelineDataSourceCreatorFactory.java  | 11 ++--
 .../scenario/rulealtered/RuleAlteredJobWorker.java | 17 +++---
 .../mysql/ingest/MySQLIncrementalDumper.java       |  7 +--
 .../mysql/ingest/column/value/ValueHandler.java    |  4 +-
 .../api/config/rulealtered/JobConfiguration.java   |  6 ---
 .../spi/rulealtered/RuleAlteredDetector.java       |  3 +-
 .../RuleAlteredJobConfigurationPreparer.java       |  3 +-
 .../jdbc/connection/JDBCBackendConnection.java     | 12 +----
 .../StatementMemoryStrictlyFetchSizeSetter.java    |  3 +-
 .../spi/required/RequiredSPIRegistry.java          |  9 +++-
 .../shardingsphere/spi/singleton/SingletonSPI.java | 23 ++------
 .../spi/singleton/SingletonSPIRegistry.java        | 61 ++++++++++++++++++++++
 .../required/RequiredSingletonSPIFixture.java      | 24 ++-------
 .../required/RequiredSingletonSPIFixtureImpl.java  | 27 +++-------
 .../spi/fixture/singleton/SingletonSPIFixture.java | 20 ++-----
 .../fixture/singleton/SingletonSPIFixtureImpl.java | 27 +++-------
 .../spi/required/RequiredSPIRegistryTest.java      | 13 +++++
 .../spi/singleton/SingletonSPIRegistryTest.java    | 45 ++++++++++++++++
 ...pi.fixture.required.RequiredSingletonSPIFixture | 18 +++++++
 ...phere.spi.fixture.singleton.SingletonSPIFixture | 18 +++++++
 30 files changed, 244 insertions(+), 205 deletions(-)

diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderEngine.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderEngine.java
index 9429b70..35d491a 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderEngine.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/TableMetaDataLoaderEngine.java
@@ -26,7 +26,7 @@ import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.metadata.schema.builder.loader.common.TableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableMetaDataLoader;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 
 import java.sql.SQLException;
 import java.util.Collection;
@@ -44,10 +44,9 @@ import java.util.concurrent.TimeUnit;
 @Slf4j
 public final class TableMetaDataLoaderEngine {
     
-    static {
-        ShardingSphereServiceLoader.register(DialectTableMetaDataLoader.class);
-    }
-
+    private static final Map<String, DialectTableMetaDataLoader> DIALECT_METADATA_LOADER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            DialectTableMetaDataLoader.class, DialectTableMetaDataLoader::getDatabaseType);
+    
     private static final ExecutorService EXECUTOR_SERVICE = new ThreadPoolExecutor(Runtime.getRuntime().availableProcessors() * 2, Runtime.getRuntime().availableProcessors() * 2,
             0L, TimeUnit.MILLISECONDS, new LinkedBlockingQueue<>(), new ThreadFactoryBuilder().setDaemon(true).setNameFormat("ShardingSphere-TableMetaDataLoaderEngine-%d").build());
     
@@ -102,11 +101,6 @@ public final class TableMetaDataLoaderEngine {
     }
 
     private static Optional<DialectTableMetaDataLoader> findDialectTableMetaDataLoader(final DatabaseType databaseType) {
-        for (DialectTableMetaDataLoader each : ShardingSphereServiceLoader.getSingletonServiceInstances(DialectTableMetaDataLoader.class)) {
-            if (each.getDatabaseType().equals(databaseType.getName())) {
-                return Optional.of(each);
-            }
-        }
-        return Optional.empty();
+        return Optional.ofNullable(DIALECT_METADATA_LOADER_MAP.get(databaseType.getName()));
     }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/spi/DialectTableMetaDataLoader.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/spi/DialectTableMetaDataLoader.java
index f31e2fd..ff5e994 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/spi/DialectTableMetaDataLoader.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/metadata/schema/builder/spi/DialectTableMetaDataLoader.java
@@ -19,6 +19,7 @@ package org.apache.shardingsphere.infra.metadata.schema.builder.spi;
 
 import org.apache.shardingsphere.infra.database.type.DatabaseTypeAwareSPI;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
@@ -28,7 +29,7 @@ import java.util.Map;
 /**
  * Dialect table meta data loader.
  */
-public interface DialectTableMetaDataLoader extends DatabaseTypeAwareSPI {
+public interface DialectTableMetaDataLoader extends DatabaseTypeAwareSPI, SingletonSPI {
     
     /**
      * Load table meta data.
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java
index 0be6daa..1ddb357 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/MySQLTableMetaDataLoaderTest.java
@@ -21,8 +21,7 @@ import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableM
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.junit.BeforeClass;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.junit.Test;
 
 import javax.sql.DataSource;
@@ -39,10 +38,8 @@ import static org.mockito.Mockito.when;
 
 public final class MySQLTableMetaDataLoaderTest {
     
-    @BeforeClass
-    public static void setUp() {
-        ShardingSphereServiceLoader.register(DialectTableMetaDataLoader.class);
-    }
+    private static final Map<String, DialectTableMetaDataLoader> DIALECT_METADATA_LOADER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            DialectTableMetaDataLoader.class, DialectTableMetaDataLoader::getDatabaseType);
     
     @Test
     public void assertLoadWithoutTables() throws SQLException {
@@ -108,10 +105,9 @@ public final class MySQLTableMetaDataLoaderTest {
     }
 
     private DialectTableMetaDataLoader getTableMetaDataLoader() {
-        for (DialectTableMetaDataLoader each : ShardingSphereServiceLoader.getSingletonServiceInstances(DialectTableMetaDataLoader.class)) {
-            if ("MySQL".equals(each.getDatabaseType())) {
-                return each;
-            }
+        DialectTableMetaDataLoader result = DIALECT_METADATA_LOADER_MAP.get("MySQL");
+        if (null != result) {
+            return result;
         }
         throw new IllegalStateException("Can not find MySQLTableMetaDataLoader");
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OracleTableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OracleTableMetaDataLoaderTest.java
index 3648e47..c0f5abc 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OracleTableMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/OracleTableMetaDataLoaderTest.java
@@ -21,8 +21,7 @@ import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableM
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.junit.BeforeClass;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.junit.Test;
 
 import javax.sql.DataSource;
@@ -72,10 +71,8 @@ public final class OracleTableMetaDataLoaderTest {
     private static final String ALL_TAB_COLUMNS_SQL_CONDITION6 = "SELECT OWNER AS TABLE_SCHEMA, TABLE_NAME, COLUMN_NAME, DATA_TYPE, COLUMN_ID  FROM ALL_TAB_COLUMNS"
             + " WHERE OWNER = ? AND TABLE_NAME IN ('tbl') ORDER BY COLUMN_ID";
     
-    @BeforeClass
-    public static void setUp() {
-        ShardingSphereServiceLoader.register(DialectTableMetaDataLoader.class);
-    }
+    private static final Map<String, DialectTableMetaDataLoader> DIALECT_METADATA_LOADER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            DialectTableMetaDataLoader.class, DialectTableMetaDataLoader::getDatabaseType);
     
     @Test
     public void assertLoadCondition1() throws SQLException {
@@ -222,10 +219,9 @@ public final class OracleTableMetaDataLoaderTest {
     }
     
     private DialectTableMetaDataLoader getTableMetaDataLoader() {
-        for (DialectTableMetaDataLoader each : ShardingSphereServiceLoader.getSingletonServiceInstances(DialectTableMetaDataLoader.class)) {
-            if ("Oracle".equals(each.getDatabaseType())) {
-                return each;
-            }
+        DialectTableMetaDataLoader result = DIALECT_METADATA_LOADER_MAP.get("Oracle");
+        if (null != result) {
+            return result;
         }
         throw new IllegalStateException("Can not find OracleTableMetaDataLoader");
     }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/SQLServerTableMetaDataLoaderTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/SQLServerTableMetaDataLoaderTest.java
index 85504c3..f93e78d 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/SQLServerTableMetaDataLoaderTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/metadata/schema/builder/loader/dialect/SQLServerTableMetaDataLoaderTest.java
@@ -21,8 +21,7 @@ import org.apache.shardingsphere.infra.metadata.schema.builder.spi.DialectTableM
 import org.apache.shardingsphere.infra.metadata.schema.model.ColumnMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.IndexMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.junit.BeforeClass;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.junit.Test;
 
 import javax.sql.DataSource;
@@ -39,10 +38,8 @@ import static org.mockito.Mockito.when;
 
 public final class SQLServerTableMetaDataLoaderTest {
     
-    @BeforeClass
-    public static void setUp() {
-        ShardingSphereServiceLoader.register(DialectTableMetaDataLoader.class);
-    }
+    private static final Map<String, DialectTableMetaDataLoader> DIALECT_METADATA_LOADER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            DialectTableMetaDataLoader.class, DialectTableMetaDataLoader::getDatabaseType);
     
     @Test
     public void assertLoadWithoutTables() throws SQLException {
@@ -119,10 +116,9 @@ public final class SQLServerTableMetaDataLoaderTest {
     }
     
     private DialectTableMetaDataLoader getTableMetaDataLoader() {
-        for (DialectTableMetaDataLoader each : ShardingSphereServiceLoader.getSingletonServiceInstances(DialectTableMetaDataLoader.class)) {
-            if ("SQLServer".equals(each.getDatabaseType())) {
-                return each;
-            }
+        DialectTableMetaDataLoader result = DIALECT_METADATA_LOADER_MAP.get("SQLServer");
+        if (null != result) {
+            return result;
         }
         throw new IllegalStateException("Can not find SQLServerTableMetaDataLoader");
     }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java
index ae473e7..eee83a7 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java
@@ -18,6 +18,7 @@
 package org.apache.shardingsphere.driver.state;
 
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.apache.shardingsphere.spi.typed.TypedSPI;
 
 import java.sql.Connection;
@@ -25,7 +26,7 @@ import java.sql.Connection;
 /**
  * Driver state.
  */
-public interface DriverState extends TypedSPI {
+public interface DriverState extends TypedSPI, SingletonSPI {
     
     /**
      * Get connection.
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverStateContext.java b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverStateContext.java
index f7c4a19..4bcd1d1 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverStateContext.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverStateContext.java
@@ -20,11 +20,9 @@ package org.apache.shardingsphere.driver.state;
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.mode.manager.ContextManager;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 
 import java.sql.Connection;
-import java.util.Collection;
-import java.util.HashMap;
 import java.util.Map;
 
 /**
@@ -36,13 +34,7 @@ public final class DriverStateContext {
     private static final Map<String, DriverState> STATES;
     
     static {
-        // TODO add singleton cache with TypedSPI init
-        ShardingSphereServiceLoader.register(DriverState.class);
-        Collection<DriverState> driverStates = ShardingSphereServiceLoader.getSingletonServiceInstances(DriverState.class);
-        STATES = new HashMap<>();
-        for (DriverState each : driverStates) {
-            STATES.put(each.getType(), each);
-        }
+        STATES = SingletonSPIRegistry.getTypedSingletonInstancesMap(DriverState.class);
     }
     
     /**
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/DataSourcePropertiesSetter.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/DataSourcePropertiesSetter.java
index 1e20645..30d6bd9 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/DataSourcePropertiesSetter.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/DataSourcePropertiesSetter.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.spring.boot.datasource.prop;
 
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.springframework.core.env.Environment;
 
 import javax.sql.DataSource;
@@ -24,7 +25,7 @@ import javax.sql.DataSource;
 /**
  * Different datasource properties setter.
  */
-public interface DataSourcePropertiesSetter {
+public interface DataSourcePropertiesSetter extends SingletonSPI {
     
     /**
      * Set datasource custom properties.
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/impl/DataSourcePropertiesSetterHolder.java b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/impl/DataSourcePropertiesSetterHolder.java
index 27859ff..eb550aa 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/impl/DataSourcePropertiesSetterHolder.java
+++ b/shardingsphere-jdbc/shardingsphere-jdbc-spring/shardingsphere-jdbc-spring-infra/shardingsphere-jdbc-spring-boot-starter-infra/src/main/java/org/apache/shardingsphere/spring/boot/datasource/prop/impl/DataSourcePropertiesSetterHolder.java
@@ -19,10 +19,9 @@ package org.apache.shardingsphere.spring.boot.datasource.prop.impl;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.apache.shardingsphere.spring.boot.datasource.prop.DataSourcePropertiesSetter;
 
-import java.util.HashMap;
 import java.util.Map;
 import java.util.Optional;
 
@@ -32,13 +31,10 @@ import java.util.Optional;
 @NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class DataSourcePropertiesSetterHolder {
     
-    private static final Map<String, DataSourcePropertiesSetter> DATA_SOURCE_PROPERTIES_SETTER_MAP = new HashMap<>();
+    private static final Map<String, DataSourcePropertiesSetter> DATA_SOURCE_PROPERTIES_SETTER_MAP;
     
     static {
-        ShardingSphereServiceLoader.register(DataSourcePropertiesSetter.class);
-        for (DataSourcePropertiesSetter each : ShardingSphereServiceLoader.getSingletonServiceInstances(DataSourcePropertiesSetter.class)) {
-            DATA_SOURCE_PROPERTIES_SETTER_MAP.put(each.getType(), each);
-        }
+        DATA_SOURCE_PROPERTIES_SETTER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(DataSourcePropertiesSetter.class, DataSourcePropertiesSetter::getType);
     }
     
     /**
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreator.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreator.java
index 5bdcb06..188d0b4 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreator.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreator.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.data.pipeline.core.datasource.creator;
 
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.apache.shardingsphere.spi.typed.TypedSPI;
 
 import javax.sql.DataSource;
@@ -25,7 +26,7 @@ import java.sql.SQLException;
 /**
  * Pipeline data source creator.
  */
-public interface PipelineDataSourceCreator extends TypedSPI {
+public interface PipelineDataSourceCreator extends TypedSPI, SingletonSPI {
     
     /**
      * Create pipeline data source.
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreatorFactory.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreatorFactory.java
index 0a8f88f..aed8103 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreatorFactory.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/core/datasource/creator/PipelineDataSourceCreatorFactory.java
@@ -17,17 +17,16 @@
 
 package org.apache.shardingsphere.data.pipeline.core.datasource.creator;
 
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.typed.TypedSPIRegistry;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
+
+import java.util.Map;
 
 /**
  * Pipeline data source creator factory.
  */
 public final class PipelineDataSourceCreatorFactory {
     
-    static {
-        ShardingSphereServiceLoader.register(PipelineDataSourceCreator.class);
-    }
+    private static final Map<String, PipelineDataSourceCreator> DATA_SOURCE_CREATOR_MAP = SingletonSPIRegistry.getTypedSingletonInstancesMap(PipelineDataSourceCreator.class);
     
     /**
      * Get pipeline data source creator instance.
@@ -36,6 +35,6 @@ public final class PipelineDataSourceCreatorFactory {
      * @return pipeline data source creator instance
      */
     public static PipelineDataSourceCreator getInstance(final String type) {
-        return TypedSPIRegistry.getRegisteredService(PipelineDataSourceCreator.class, type, null);
+        return DATA_SOURCE_CREATOR_MAP.get(type);
     }
 }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/rulealtered/RuleAlteredJobWorker.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/rulealtered/RuleAlteredJobWorker.java
index faf5a25..475f857 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/rulealtered/RuleAlteredJobWorker.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-core/src/main/java/org/apache/shardingsphere/data/pipeline/scenario/rulealtered/RuleAlteredJobWorker.java
@@ -44,7 +44,7 @@ import org.apache.shardingsphere.infra.yaml.config.swapper.YamlRuleConfiguration
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.cache.event.StartScalingEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.rule.ScalingTaskFinishedEvent;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -65,19 +65,14 @@ import java.util.stream.Collectors;
 @Slf4j
 public final class RuleAlteredJobWorker {
     
-    static {
-        ShardingSphereServiceLoader.register(RuleAlteredDetector.class);
-    }
-    
     private static final RuleAlteredJobWorker INSTANCE = new RuleAlteredJobWorker();
     
-    // TODO refactor
-    private static final Map<String, RuleAlteredDetector> RULE_CLASS_NAME_DETECTOR_MAP = ShardingSphereServiceLoader.getSingletonServiceInstances(RuleAlteredDetector.class).stream()
-            .collect(Collectors.toMap(RuleAlteredDetector::getRuleConfigClassName, Function.identity()));
-    
-    private static final Map<String, RuleAlteredDetector> YAML_RULE_CLASS_NAME_DETECTOR_MAP = ShardingSphereServiceLoader.getSingletonServiceInstances(RuleAlteredDetector.class).stream()
-            .collect(Collectors.toMap(RuleAlteredDetector::getYamlRuleConfigClassName, Function.identity()));
+    private static final Map<String, RuleAlteredDetector> RULE_CLASS_NAME_DETECTOR_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            RuleAlteredDetector.class, RuleAlteredDetector::getRuleConfigClassName);
     
+    private static final Map<String, RuleAlteredDetector> YAML_RULE_CLASS_NAME_DETECTOR_MAP = SingletonSPIRegistry.getSingletonInstancesMap(
+            RuleAlteredDetector.class, RuleAlteredDetector::getYamlRuleConfigClassName);
+        
     private static final YamlRuleConfigurationSwapperEngine SWAPPER_ENGINE = new YamlRuleConfigurationSwapperEngine();
     
     private static final AtomicBoolean WORKER_INITIALIZED = new AtomicBoolean(false);
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLIncrementalDumper.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLIncrementalDumper.java
index f35655d..4696b3c 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLIncrementalDumper.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/MySQLIncrementalDumper.java
@@ -49,7 +49,7 @@ import org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value.ValueHa
 import org.apache.shardingsphere.data.pipeline.spi.ingest.dumper.IncrementalDumper;
 import org.apache.shardingsphere.infra.database.metadata.url.JdbcUrl;
 import org.apache.shardingsphere.infra.database.metadata.url.StandardJdbcUrlParser;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 
 import java.io.Serializable;
 import java.security.SecureRandom;
@@ -57,7 +57,6 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Random;
-import java.util.stream.Collectors;
 
 /**
  * MySQL incremental dumper.
@@ -79,9 +78,7 @@ public final class MySQLIncrementalDumper extends AbstractLifecycleExecutor impl
     private PipelineChannel channel;
     
     static {
-        ShardingSphereServiceLoader.register(ValueHandler.class);
-        VALUE_HANDLER_MAP = ShardingSphereServiceLoader.getSingletonServiceInstances(ValueHandler.class)
-                .stream().collect(Collectors.toMap(ValueHandler::getTypeName, v -> v));
+        VALUE_HANDLER_MAP = SingletonSPIRegistry.getSingletonInstancesMap(ValueHandler.class, ValueHandler::getTypeName);
     }
     
     public MySQLIncrementalDumper(final DumperConfiguration dumperConfig, final IngestPosition<BinlogPosition> binlogPosition) {
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
index 7356a8e..340ed27 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
@@ -17,12 +17,14 @@
 
 package org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value;
 
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
+
 import java.io.Serializable;
 
 /**
  * Value handler.
  */
-public interface ValueHandler {
+public interface ValueHandler extends SingletonSPI {
     
     /**
      * Get support type name.
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/api/config/rulealtered/JobConfiguration.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/api/config/rulealtered/JobConfiguration.java
index 517df95..3a7dd5b 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/api/config/rulealtered/JobConfiguration.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/api/config/rulealtered/JobConfiguration.java
@@ -26,7 +26,6 @@ import lombok.extern.slf4j.Slf4j;
 import org.apache.shardingsphere.data.pipeline.api.datasource.config.PipelineDataSourceConfiguration;
 import org.apache.shardingsphere.data.pipeline.api.datasource.config.PipelineDataSourceConfigurationFactory;
 import org.apache.shardingsphere.data.pipeline.spi.rulealtered.RuleAlteredJobConfigurationPreparer;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.spi.required.RequiredSPIRegistry;
 
 import java.util.Collection;
@@ -43,10 +42,6 @@ import java.util.concurrent.ThreadLocalRandom;
 // TODO share for totally new scenario
 public final class JobConfiguration {
     
-    static {
-        ShardingSphereServiceLoader.register(RuleAlteredJobConfigurationPreparer.class);
-    }
-    
     private WorkflowConfiguration workflowConfig;
     
     private PipelineConfiguration pipelineConfig;
@@ -65,7 +60,6 @@ public final class JobConfiguration {
         PipelineConfiguration pipelineConfig = getPipelineConfig();
         HandleConfiguration handleConfig = getHandleConfig();
         if (null == handleConfig || null == handleConfig.getJobShardingDataNodes()) {
-            // TODO singleton
             RuleAlteredJobConfigurationPreparer preparer = RequiredSPIRegistry.getRegisteredService(RuleAlteredJobConfigurationPreparer.class);
             handleConfig = preparer.createHandleConfiguration(pipelineConfig);
             this.handleConfig = handleConfig;
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredDetector.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredDetector.java
index a396682..c50cb9d 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredDetector.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredDetector.java
@@ -20,13 +20,14 @@ package org.apache.shardingsphere.data.pipeline.spi.rulealtered;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.config.rulealtered.OnRuleAlteredActionConfiguration;
 import org.apache.shardingsphere.infra.yaml.config.pojo.YamlRuleConfiguration;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 
 import java.util.Optional;
 
 /**
  * Rule altered detector, SPI interface.
  */
-public interface RuleAlteredDetector {
+public interface RuleAlteredDetector extends SingletonSPI {
     
     /**
      * Get YAML rule configuration class name.
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredJobConfigurationPreparer.java b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredJobConfigurationPreparer.java
index 25d031b..94a9b35 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredJobConfigurationPreparer.java
+++ b/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-spi/src/main/java/org/apache/shardingsphere/data/pipeline/spi/rulealtered/RuleAlteredJobConfigurationPreparer.java
@@ -21,13 +21,14 @@ import org.apache.shardingsphere.data.pipeline.api.config.rulealtered.HandleConf
 import org.apache.shardingsphere.data.pipeline.api.config.rulealtered.PipelineConfiguration;
 import org.apache.shardingsphere.data.pipeline.api.config.rulealtered.TaskConfiguration;
 import org.apache.shardingsphere.spi.required.RequiredSPI;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 
 import java.util.Collection;
 
 /**
  * Rule altered job configuration preparer.
  */
-public interface RuleAlteredJobConfigurationPreparer extends RequiredSPI {
+public interface RuleAlteredJobConfigurationPreparer extends RequiredSPI, SingletonSPI {
     
     /**
      * Create handle configuration, used to build job configuration.
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
index feece98..206faa9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/connection/JDBCBackendConnection.java
@@ -35,8 +35,7 @@ import org.apache.shardingsphere.proxy.backend.communication.jdbc.transaction.JD
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.BackendConnectionException;
 import org.apache.shardingsphere.proxy.backend.session.ConnectionSession;
-import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
-import org.apache.shardingsphere.spi.typed.TypedSPI;
+import org.apache.shardingsphere.spi.singleton.SingletonSPIRegistry;
 import org.apache.shardingsphere.transaction.core.TransactionType;
 
 import java.sql.Connection;
@@ -51,8 +50,6 @@ import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
-import java.util.function.Function;
-import java.util.stream.Collectors;
 
 /**
  * JDBC backend connection.
@@ -61,10 +58,6 @@ import java.util.stream.Collectors;
 @Setter
 public final class JDBCBackendConnection implements BackendConnection<Void>, ExecutorJDBCManager {
     
-    static {
-        ShardingSphereServiceLoader.register(StatementMemoryStrictlyFetchSizeSetter.class);
-    }
-    
     private final ConnectionSession connectionSession;
     
     private volatile FederationExecutor federationExecutor;
@@ -85,8 +78,7 @@ public final class JDBCBackendConnection implements BackendConnection<Void>, Exe
     
     public JDBCBackendConnection(final ConnectionSession connectionSession) {
         this.connectionSession = connectionSession;
-        fetchSizeSetters = ShardingSphereServiceLoader.getSingletonServiceInstances(StatementMemoryStrictlyFetchSizeSetter.class).stream()
-                .collect(Collectors.toMap(TypedSPI::getType, Function.identity()));
+        fetchSizeSetters = SingletonSPIRegistry.getTypedSingletonInstancesMap(StatementMemoryStrictlyFetchSizeSetter.class);
     }
     
     @Override
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
index d578ea8..bf73682 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/communication/jdbc/statement/StatementMemoryStrictlyFetchSizeSetter.java
@@ -17,6 +17,7 @@
 
 package org.apache.shardingsphere.proxy.backend.communication.jdbc.statement;
 
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.apache.shardingsphere.spi.typed.TypedSPI;
 
 import java.sql.SQLException;
@@ -25,7 +26,7 @@ import java.sql.Statement;
 /**
  * Statement memory strictly fetch size setter.
  */
-public interface StatementMemoryStrictlyFetchSizeSetter extends TypedSPI {
+public interface StatementMemoryStrictlyFetchSizeSetter extends TypedSPI, SingletonSPI {
     
     /**
      * Set fetch size.
diff --git a/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistry.java b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistry.java
index 51820f2..6fe3e8f 100644
--- a/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistry.java
+++ b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistry.java
@@ -21,6 +21,7 @@ import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
 import org.apache.shardingsphere.spi.exception.ServiceProviderNotFoundException;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 
 import java.util.Collection;
 
@@ -38,7 +39,13 @@ public final class RequiredSPIRegistry {
      * @return registered service
      */
     public static <T extends RequiredSPI> T getRegisteredService(final Class<T> requiredSPIClass) {
-        Collection<T> services = ShardingSphereServiceLoader.newServiceInstances(requiredSPIClass);
+        Collection<T> services;
+        if (SingletonSPI.class.isAssignableFrom(requiredSPIClass)) {
+            ShardingSphereServiceLoader.register(requiredSPIClass);
+            services = ShardingSphereServiceLoader.getSingletonServiceInstances(requiredSPIClass);
+        } else {
+            services = ShardingSphereServiceLoader.newServiceInstances(requiredSPIClass);
+        }
         if (services.isEmpty()) {
             throw new ServiceProviderNotFoundException(requiredSPIClass);
         }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPI.java
similarity index 65%
copy from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
copy to shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPI.java
index 7356a8e..7e3ea32 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
+++ b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPI.java
@@ -15,27 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value;
-
-import java.io.Serializable;
+package org.apache.shardingsphere.spi.singleton;
 
 /**
- * Value handler.
+ * Singleton SPI.
  */
-public interface ValueHandler {
-    
-    /**
-     * Get support type name.
-     *
-     * @return type name
-     */
-    String getTypeName();
-    
-    /**
-     * Handle column value.
-     *
-     * @param value column value
-     * @return handled column value
-     */
-    Serializable handle(Serializable value);
+public interface SingletonSPI {
 }
diff --git a/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistry.java b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistry.java
new file mode 100644
index 0000000..21f028b
--- /dev/null
+++ b/shardingsphere-spi/src/main/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistry.java
@@ -0,0 +1,61 @@
+/*
+ * 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.spi.singleton;
+
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.spi.typed.TypedSPI;
+
+import java.util.Collection;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Singleton SPI registry.
+ */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
+public final class SingletonSPIRegistry {
+    
+    /**
+     * Get singleton instances map.
+     *
+     * @param singletonSPIClass singleton SPI class
+     * @param keyMapper key mapper
+     * @param <K> the output type of the key mapping function
+     * @param <T> the type of the input elements
+     * @return singleton instances map
+     */
+    public static <K, T> Map<K, T> getSingletonInstancesMap(final Class<T> singletonSPIClass, final Function<? super T, ? extends K> keyMapper) {
+        ShardingSphereServiceLoader.register(singletonSPIClass);
+        Collection<T> instances = ShardingSphereServiceLoader.getSingletonServiceInstances(singletonSPIClass);
+        return instances.stream().collect(Collectors.toMap(keyMapper, Function.identity()));
+    }
+    
+    /**
+     * Get typed singleton instances map.
+     *
+     * @param singletonSPIClass singleton SPI class
+     * @param <T> the type of the input elements
+     * @return singleton instances map
+     */
+    public static <T extends TypedSPI> Map<String, T> getTypedSingletonInstancesMap(final Class<T> singletonSPIClass) {
+        return getSingletonInstancesMap(singletonSPIClass, TypedSPI::getType);
+    }
+}
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixture.java
similarity index 64%
copy from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
copy to shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixture.java
index 7356a8e..35d25fa 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixture.java
@@ -15,27 +15,9 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value;
+package org.apache.shardingsphere.spi.fixture.required;
 
-import java.io.Serializable;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 
-/**
- * Value handler.
- */
-public interface ValueHandler {
-    
-    /**
-     * Get support type name.
-     *
-     * @return type name
-     */
-    String getTypeName();
-    
-    /**
-     * Handle column value.
-     *
-     * @param value column value
-     * @return handled column value
-     */
-    Serializable handle(Serializable value);
+public interface RequiredSingletonSPIFixture extends RequiredSPIFixture, SingletonSPI {
 }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixtureImpl.java
similarity index 64%
copy from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
copy to shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixtureImpl.java
index 7356a8e..dd79171 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/required/RequiredSingletonSPIFixtureImpl.java
@@ -15,27 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value;
+package org.apache.shardingsphere.spi.fixture.required;
 
-import java.io.Serializable;
-
-/**
- * Value handler.
- */
-public interface ValueHandler {
-    
-    /**
-     * Get support type name.
-     *
-     * @return type name
-     */
-    String getTypeName();
+public final class RequiredSingletonSPIFixtureImpl implements RequiredSingletonSPIFixture {
     
-    /**
-     * Handle column value.
-     *
-     * @param value column value
-     * @return handled column value
-     */
-    Serializable handle(Serializable value);
+    @Override
+    public boolean isDefault() {
+        return false;
+    }
 }
diff --git a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixture.java
similarity index 65%
copy from shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java
copy to shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixture.java
index ae473e7..c477d56 100644
--- a/shardingsphere-jdbc/shardingsphere-jdbc-core/src/main/java/org/apache/shardingsphere/driver/state/DriverState.java
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixture.java
@@ -15,24 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.driver.state;
+package org.apache.shardingsphere.spi.fixture.singleton;
 
-import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.spi.singleton.SingletonSPI;
 import org.apache.shardingsphere.spi.typed.TypedSPI;
 
-import java.sql.Connection;
-
-/**
- * Driver state.
- */
-public interface DriverState extends TypedSPI {
-    
-    /**
-     * Get connection.
-     *
-     * @param schemaName schema name
-     * @param contextManager context manager
-     * @return connection
-     */
-    Connection getConnection(String schemaName, ContextManager contextManager);
+public interface SingletonSPIFixture extends SingletonSPI, TypedSPI {
 }
diff --git a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixtureImpl.java
similarity index 64%
copy from shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
copy to shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixtureImpl.java
index 7356a8e..784c43d 100644
--- a/shardingsphere-kernel/shardingsphere-data-pipeline/shardingsphere-data-pipeline-dialect/shardingsphere-data-pipeline-mysql/src/main/java/org/apache/shardingsphere/data/pipeline/mysql/ingest/column/value/ValueHandler.java
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/fixture/singleton/SingletonSPIFixtureImpl.java
@@ -15,27 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.data.pipeline.mysql.ingest.column.value;
+package org.apache.shardingsphere.spi.fixture.singleton;
 
-import java.io.Serializable;
-
-/**
- * Value handler.
- */
-public interface ValueHandler {
-    
-    /**
-     * Get support type name.
-     *
-     * @return type name
-     */
-    String getTypeName();
+public final class SingletonSPIFixtureImpl implements SingletonSPIFixture {
     
-    /**
-     * Handle column value.
-     *
-     * @param value column value
-     * @return handled column value
-     */
-    Serializable handle(Serializable value);
+    @Override
+    public String getType() {
+        return "SINGLETON_FIXTURE";
+    }
 }
diff --git a/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistryTest.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistryTest.java
index 39a4b7b..0d08e73 100644
--- a/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistryTest.java
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/required/RequiredSPIRegistryTest.java
@@ -23,8 +23,11 @@ import org.apache.shardingsphere.spi.fixture.required.NoImplRequiredSPIFixture;
 import org.apache.shardingsphere.spi.fixture.required.RequiredSPIFixture;
 import org.apache.shardingsphere.spi.fixture.required.RequiredSPIFixtureDefaultTrueImpl;
 import org.apache.shardingsphere.spi.fixture.required.RequiredSPIImpl;
+import org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixture;
+import org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixtureImpl;
 import org.junit.Test;
 
+import static org.junit.Assert.assertSame;
 import static org.junit.Assert.assertTrue;
 
 public final class RequiredSPIRegistryTest {
@@ -32,6 +35,7 @@ public final class RequiredSPIRegistryTest {
     static {
         ShardingSphereServiceLoader.register(RequiredSPIFixture.class);
         ShardingSphereServiceLoader.register(RequiredSPI.class);
+        ShardingSphereServiceLoader.register(RequiredSingletonSPIFixture.class);
     }
     
     @Test(expected = ServiceProviderNotFoundException.class)
@@ -50,4 +54,13 @@ public final class RequiredSPIRegistryTest {
         RequiredSPIFixture actualRegisteredService = RequiredSPIRegistry.getRegisteredService(RequiredSPIFixture.class);
         assertTrue(actualRegisteredService instanceof RequiredSPIFixtureDefaultTrueImpl);
     }
+    
+    @Test
+    public void assertRegisteredServiceSingleton() {
+        RequiredSPIFixture actualOne = RequiredSPIRegistry.getRegisteredService(RequiredSingletonSPIFixture.class);
+        assertTrue(actualOne instanceof RequiredSingletonSPIFixtureImpl);
+        RequiredSPIFixture actualTwo = RequiredSPIRegistry.getRegisteredService(RequiredSingletonSPIFixture.class);
+        assertTrue(actualTwo instanceof RequiredSingletonSPIFixtureImpl);
+        assertSame(actualOne, actualTwo);
+    }
 }
diff --git a/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistryTest.java b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistryTest.java
new file mode 100644
index 0000000..e2a6249
--- /dev/null
+++ b/shardingsphere-spi/src/test/java/org/apache/shardingsphere/spi/singleton/SingletonSPIRegistryTest.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.spi.singleton;
+
+import org.apache.shardingsphere.spi.fixture.singleton.SingletonSPIFixture;
+import org.junit.Test;
+
+import java.util.Map;
+
+import static org.hamcrest.CoreMatchers.is;
+import static org.junit.Assert.assertThat;
+import static org.junit.Assert.assertTrue;
+
+public final class SingletonSPIRegistryTest {
+    
+    @Test
+    public void assertGetSingletonInstancesMap() {
+        Map<String, SingletonSPIFixture> singletonSPIFixtureMap = SingletonSPIRegistry.getSingletonInstancesMap(
+                SingletonSPIFixture.class, SingletonSPIFixture::getType);
+        assertThat(singletonSPIFixtureMap.size(), is(1));
+        assertTrue(singletonSPIFixtureMap.containsKey("SINGLETON_FIXTURE"));
+    }
+    
+    @Test
+    public void assertGetTypedSingletonInstancesMap() {
+        Map<String, SingletonSPIFixture> singletonSPIFixtureMap = SingletonSPIRegistry.getTypedSingletonInstancesMap(SingletonSPIFixture.class);
+        assertThat(singletonSPIFixtureMap.size(), is(1));
+        assertTrue(singletonSPIFixtureMap.containsKey("SINGLETON_FIXTURE"));
+    }
+}
diff --git a/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixture b/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixture
new file mode 100644
index 0000000..9ae056d
--- /dev/null
+++ b/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixture
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+org.apache.shardingsphere.spi.fixture.required.RequiredSingletonSPIFixtureImpl
diff --git a/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.singleton.SingletonSPIFixture b/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.singleton.SingletonSPIFixture
new file mode 100644
index 0000000..74ed540
--- /dev/null
+++ b/shardingsphere-spi/src/test/resources/META-INF/services/org.apache.shardingsphere.spi.fixture.singleton.SingletonSPIFixture
@@ -0,0 +1,18 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements.  See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License.  You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+org.apache.shardingsphere.spi.fixture.singleton.SingletonSPIFixtureImpl