You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by xi...@apache.org on 2020/12/11 11:42:28 UTC

[shardingsphere] branch master updated: Initialize lock strategy with SPI (#8578)

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

xiaoyu 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 9f8e52e  Initialize lock strategy with SPI (#8578)
9f8e52e is described below

commit 9f8e52e1e1799e2d1ac678f3b38df9b034f97d1d
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Fri Dec 11 19:41:56 2020 +0800

    Initialize lock strategy with SPI (#8578)
---
 .../governance/core/facade/GovernanceFacade.java   |  4 +-
 .../core/lock/GovernanceLockStrategy.java          | 84 ----------------------
 .../governance/core/lock/LockCenter.java           | 77 +++++++++++++++++---
 .../core/lock/strategy/GovernanceLockStrategy.java | 39 ++++------
 ...g.apache.shardingsphere.infra.lock.LockStrategy | 18 +++++
 .../governance/core/lock/LockCenterTest.java       |  4 +-
 .../{ => strategy}/GovernanceLockStrategyTest.java | 13 ++--
 .../shardingsphere/infra/lock/LockContext.java     | 15 ++--
 .../shardingsphere/infra/lock/LockStrategy.java    |  4 +-
 .../{LockStrategy.java => LockStrategyType.java}   | 24 +------
 .../infra/lock/StandardLockStrategy.java           |  5 ++
 ...g.apache.shardingsphere.infra.lock.LockStrategy | 18 +++++
 .../shardingsphere/infra/lock/LockContextTest.java |  7 +-
 .../infra/lock/fixture/FixtureLockStrategy.java}   | 43 ++++++-----
 ...g.apache.shardingsphere.infra.lock.LockStrategy | 18 +++++
 .../impl/GovernanceBootstrapInitializer.java       |  4 +-
 .../impl/StandardBootstrapInitializer.java         |  4 +-
 17 files changed, 195 insertions(+), 186 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/facade/GovernanceFacade.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/facade/GovernanceFacade.java
index fb15eb4..b5363f4 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/facade/GovernanceFacade.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/facade/GovernanceFacade.java
@@ -52,7 +52,7 @@ public final class GovernanceFacade implements AutoCloseable {
     private GovernanceListenerManager listenerManager;
     
     @Getter
-    private LockCenter lockCenter;
+    private LockCenter lockCenter = LockCenter.getInstance();
     
     /**
      * Initialize governance facade.
@@ -65,7 +65,7 @@ public final class GovernanceFacade implements AutoCloseable {
         repositoryFacade = new GovernanceRepositoryFacade(config);
         registryCenter = new RegistryCenter(repositoryFacade.getRegistryRepository());
         configCenter = new ConfigCenter(repositoryFacade.getConfigurationRepository());
-        lockCenter = new LockCenter(repositoryFacade.getRegistryRepository(), registryCenter);
+        lockCenter.init(repositoryFacade.getRegistryRepository(), registryCenter);
         listenerManager = new GovernanceListenerManager(repositoryFacade.getRegistryRepository(),
                 repositoryFacade.getConfigurationRepository(), schemaNames.isEmpty() ? configCenter.getAllSchemaNames() : schemaNames);
         GovernedStateContext.startUp();
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategy.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategy.java
deleted file mode 100644
index ba5b3ad..0000000
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategy.java
+++ /dev/null
@@ -1,84 +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.lock;
-
-import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
-import org.apache.shardingsphere.governance.core.registry.RegistryCenterNodeStatus;
-import org.apache.shardingsphere.infra.lock.LockStrategy;
-
-import java.util.Collection;
-
-/**
- * Governance lock strategy.
- */
-public final class GovernanceLockStrategy implements LockStrategy {
-    
-    private static final int CHECK_RETRY_MAXIMUM = 5;
-    
-    private static final int CHECK_RETRY_INTERVAL_SECONDS = 3;
-    
-    private final GovernanceFacade governanceFacade;
-    
-    public GovernanceLockStrategy(final GovernanceFacade governanceFacade) {
-        this.governanceFacade = governanceFacade;
-    }
-    
-    @Override
-    public boolean tryLock(final Long timeout) {
-        return governanceFacade.getLockCenter().tryGlobalLock(timeout);
-    }
-    
-    @Override
-    public void releaseLock() {
-        governanceFacade.getLockCenter().releaseGlobalLock();
-    }
-    
-    @Override
-    public boolean checkLock() {
-        Collection<String> instanceIds = governanceFacade.getRegistryCenter().loadAllInstances();
-        if (instanceIds.isEmpty()) {
-            return true;
-        }
-        return checkOrRetry(instanceIds);
-    }
-    
-    private boolean checkOrRetry(final Collection<String> instanceIds) {
-        for (int i = 0; i < CHECK_RETRY_MAXIMUM; i++) {
-            if (check(instanceIds)) {
-                return true;
-            }
-            try {
-                Thread.sleep(CHECK_RETRY_INTERVAL_SECONDS * 1000L);
-                // CHECKSTYLE:OFF
-            } catch (final InterruptedException ex) {
-                // CHECKSTYLE:ON
-            }
-        }
-        return false;
-    }
-    
-    private boolean check(final Collection<String> instanceIds) {
-        for (String instanceId : instanceIds) {
-            if (!RegistryCenterNodeStatus.LOCKED.toString()
-                    .equalsIgnoreCase(governanceFacade.getRegistryCenter().loadInstanceData(instanceId))) {
-                return false;
-            }
-        }
-        return true;
-    }
-}
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/LockCenter.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/LockCenter.java
index 8b96d1c..2189643 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/LockCenter.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/LockCenter.java
@@ -18,35 +18,58 @@
 package org.apache.shardingsphere.governance.core.lock;
 
 import com.google.common.eventbus.Subscribe;
-import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
+import lombok.AccessLevel;
+import lombok.NoArgsConstructor;
 import org.apache.shardingsphere.governance.core.event.model.lock.GlobalLockAddedEvent;
 import org.apache.shardingsphere.governance.core.lock.node.LockNode;
 import org.apache.shardingsphere.governance.core.registry.RegistryCenter;
 import org.apache.shardingsphere.governance.core.registry.RegistryCenterNodeStatus;
 import org.apache.shardingsphere.governance.core.state.GovernedState;
 import org.apache.shardingsphere.governance.repository.api.RegistryRepository;
+import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 
+import java.util.Collection;
 import java.util.Optional;
 import java.util.concurrent.TimeUnit;
 
 /**
  * Lock center.
  */
+@NoArgsConstructor(access = AccessLevel.PRIVATE)
 public final class LockCenter {
     
-    private final RegistryRepository registryRepository;
+    private static final int CHECK_RETRY_MAXIMUM = 5;
+    
+    private static final int CHECK_RETRY_INTERVAL_SECONDS = 3;
+    
+    private static final LockCenter INSTANCE = new LockCenter();
     
-    private final RegistryCenter registryCenter;
+    private RegistryRepository registryRepository;
     
-    private final LockNode lockNode;
+    private RegistryCenter registryCenter;
     
-    private final GovernedState governedState;
+    private final LockNode lockNode = new LockNode();
     
-    public LockCenter(final RegistryRepository registryRepository, final RegistryCenter registryCenter) {
+    private final GovernedState governedState = new GovernedState();
+    
+    /**
+     * Get lock center instance.
+     * 
+     * @return lock center instance
+     */
+    public static LockCenter getInstance() {
+        return INSTANCE;
+    }
+    
+    /**
+     * Initialize lock center.
+     * 
+     * @param registryRepository registry repository
+     * @param registryCenter registry center
+     */
+    public void init(final RegistryRepository registryRepository, final RegistryCenter registryCenter) {
         this.registryRepository = registryRepository;
         this.registryCenter = registryCenter;
-        this.lockNode = new LockNode();
-        this.governedState = new GovernedState();
         this.registryRepository.initLock(lockNode.getGlobalLockNodePath());
         ShardingSphereEventBus.getInstance().register(this);
     }
@@ -89,4 +112,42 @@ public final class LockCenter {
         registryRepository.releaseLock();
         registryRepository.delete(lockNode.getGlobalLockNodePath());
     }
+    
+    /**
+     * Check lock state.
+     * 
+     * @return true if all instances were locked, else false
+     */
+    public boolean checkLock() {
+        Collection<String> instanceIds = registryCenter.loadAllInstances();
+        if (instanceIds.isEmpty()) {
+            return true;
+        }
+        return checkOrRetry(instanceIds);
+    }
+    
+    private boolean checkOrRetry(final Collection<String> instanceIds) {
+        for (int i = 0; i < CHECK_RETRY_MAXIMUM; i++) {
+            if (check(instanceIds)) {
+                return true;
+            }
+            try {
+                Thread.sleep(CHECK_RETRY_INTERVAL_SECONDS * 1000L);
+                // CHECKSTYLE:OFF
+            } catch (final InterruptedException ex) {
+                // CHECKSTYLE:ON
+            }
+        }
+        return false;
+    }
+    
+    private boolean check(final Collection<String> instanceIds) {
+        for (String instanceId : instanceIds) {
+            if (!RegistryCenterNodeStatus.LOCKED.toString()
+                    .equalsIgnoreCase(registryCenter.loadInstanceData(instanceId))) {
+                return false;
+            }
+        }
+        return true;
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategy.java
similarity index 50%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java
copy to shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategy.java
index 9395aba..04fb739 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategy.java
@@ -15,45 +15,36 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.lock;
+package org.apache.shardingsphere.governance.core.lock.strategy;
 
-import org.apache.shardingsphere.infra.state.StateContext;
-import org.apache.shardingsphere.infra.state.StateEvent;
-import org.apache.shardingsphere.infra.state.StateType;
-
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantLock;
+import org.apache.shardingsphere.governance.core.lock.LockCenter;
+import org.apache.shardingsphere.infra.lock.LockStrategy;
+import org.apache.shardingsphere.infra.lock.LockStrategyType;
 
 /**
- * Standard lock strategy.
+ * Governance lock strategy.
  */
-public final class StandardLockStrategy implements LockStrategy {
+public final class GovernanceLockStrategy implements LockStrategy {
     
-    private final ReentrantLock lock = new ReentrantLock();
+    private final LockCenter lockCenter = LockCenter.getInstance();
     
     @Override
     public boolean tryLock(final Long timeout) {
-        boolean result = false;
-        try {
-            result = lock.tryLock(timeout, TimeUnit.MILLISECONDS);
-            // CHECKSTYLE:OFF
-        } catch (final InterruptedException e) {
-            // CHECKSTYLE:ON
-        }
-        if (result) {
-            StateContext.switchState(new StateEvent(StateType.LOCK, true));
-        }
-        return result;
+        return lockCenter.tryGlobalLock(timeout);
     }
     
     @Override
     public void releaseLock() {
-        lock.unlock();
-        StateContext.switchState(new StateEvent(StateType.OK, true));
+        lockCenter.releaseGlobalLock();
     }
     
     @Override
     public boolean checkLock() {
-        return StateContext.getCurrentState() == StateType.LOCK;
+        return lockCenter.checkLock();
+    }
+    
+    @Override
+    public String getType() {
+        return LockStrategyType.GOVERNANCE.name();
     }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy b/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
new file mode 100644
index 0000000..27a4d7d
--- /dev/null
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
@@ -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.governance.core.lock.strategy.GovernanceLockStrategy
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/LockCenterTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/LockCenterTest.java
index 1ea993d..94b8e40 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/LockCenterTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/LockCenterTest.java
@@ -43,11 +43,11 @@ public final class LockCenterTest {
     @Mock
     private RegistryCenter registryCenter;
     
-    private LockCenter lockCenter;
+    private LockCenter lockCenter = LockCenter.getInstance();
     
     @Before
     public void setUp() {
-        lockCenter = new LockCenter(registryRepository, registryCenter);
+        lockCenter.init(registryRepository, registryCenter);
     }
     
     @Test
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategyTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategyTest.java
similarity index 81%
rename from shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategyTest.java
rename to shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategyTest.java
index 5f7b321..7bfb7a6 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/GovernanceLockStrategyTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/lock/strategy/GovernanceLockStrategyTest.java
@@ -15,9 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.governance.core.lock;
+package org.apache.shardingsphere.governance.core.lock.strategy;
 
-import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
+import org.apache.shardingsphere.governance.core.lock.LockCenter;
+import org.apache.shardingsphere.governance.core.registry.util.FieldUtil;
 import org.apache.shardingsphere.infra.lock.LockStrategy;
 import org.junit.Before;
 import org.junit.Test;
@@ -27,23 +28,19 @@ import org.mockito.junit.MockitoJUnitRunner;
 
 import static org.mockito.ArgumentMatchers.eq;
 import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
 
 @RunWith(MockitoJUnitRunner.class)
 public final class GovernanceLockStrategyTest {
     
     @Mock
-    private GovernanceFacade governanceFacade;
-    
-    @Mock
     private LockCenter lockCenter;
     
     private LockStrategy lockStrategy;
     
     @Before
     public void setUp() {
-        lockStrategy = new GovernanceLockStrategy(governanceFacade);
-        when(governanceFacade.getLockCenter()).thenReturn(lockCenter);
+        lockStrategy = new GovernanceLockStrategy();
+        FieldUtil.setField(lockStrategy, "lockCenter", lockCenter);
     }
     
     @Test
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockContext.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockContext.java
index 37eb48d..16a0f9b 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockContext.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockContext.java
@@ -19,7 +19,10 @@ package org.apache.shardingsphere.infra.lock;
 
 import lombok.AccessLevel;
 import lombok.NoArgsConstructor;
+import org.apache.shardingsphere.infra.spi.ShardingSphereServiceLoader;
+import org.apache.shardingsphere.infra.spi.typed.TypedSPIRegistry;
 
+import java.util.Properties;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.concurrent.locks.Condition;
@@ -38,13 +41,17 @@ public final class LockContext {
     
     private static final Condition CONDITION = LOCK.newCondition();
     
+    static {
+        ShardingSphereServiceLoader.register(LockStrategy.class);
+    }
+    
     /**
      * Init lock strategy.
-     * 
-     * @param lockStrategy lock strategy
+     *
+     * @param lockStrategyType lock strategy type
      */
-    public static void init(final LockStrategy lockStrategy) {
-        LOCK_STRATEGY.set(lockStrategy);
+    public static void init(final LockStrategyType lockStrategyType) {
+        LOCK_STRATEGY.set(TypedSPIRegistry.getRegisteredService(LockStrategy.class, lockStrategyType.name(), new Properties()));
     }
     
     /**
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
index 0c9cec2..474221b 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
@@ -17,10 +17,12 @@
 
 package org.apache.shardingsphere.infra.lock;
 
+import org.apache.shardingsphere.infra.spi.typed.TypedSPI;
+
 /**
  * Lock strategy.
  */
-public interface LockStrategy {
+public interface LockStrategy extends TypedSPI {
     
     /**
      * Try to get lock.
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategyType.java
similarity index 65%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategyType.java
index 0c9cec2..240cc12 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategyType.java
@@ -18,27 +18,9 @@
 package org.apache.shardingsphere.infra.lock;
 
 /**
- * Lock strategy.
+ * Lock strategy type.
  */
-public interface LockStrategy {
+public enum LockStrategyType {
     
-    /**
-     * Try to get lock.
-     * 
-     * @param timeout the maximum time in milliseconds to acquire lock
-     * @return true if get the lock, false if not
-     */
-    boolean tryLock(Long timeout);
-    
-    /**
-     * Release lock.
-     */
-    void releaseLock();
-    
-    /**
-     * Check lock state.
-     * 
-     * @return true if all instances were locked, else false
-     */
-    boolean checkLock();
+    STANDARD, GOVERNANCE
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java
index 9395aba..5982361 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/StandardLockStrategy.java
@@ -56,4 +56,9 @@ public final class StandardLockStrategy implements LockStrategy {
     public boolean checkLock() {
         return StateContext.getCurrentState() == StateType.LOCK;
     }
+    
+    @Override
+    public String getType() {
+        return LockStrategyType.STANDARD.name();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
new file mode 100644
index 0000000..4ea7904
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
@@ -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.infra.lock.StandardLockStrategy
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/LockContextTest.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/LockContextTest.java
index 255b586..8b9f871 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/LockContextTest.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/LockContextTest.java
@@ -22,24 +22,19 @@ import org.junit.Test;
 
 import java.util.concurrent.TimeUnit;
 
-import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertThat;
 import static org.junit.Assert.assertTrue;
 
 public final class LockContextTest {
     
-    private final LockStrategy lockStrategy = new StandardLockStrategy();
-    
     @Before
     public void init() {
-        LockContext.init(lockStrategy);
+        LockContext.init(LockStrategyType.STANDARD);
     }
     
     @Test
     public void assertGetLockStrategy() {
         assertNotNull(LockContext.getLockStrategy());
-        assertThat(LockContext.getLockStrategy(), is(lockStrategy));
     }
     
     @Test
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/fixture/FixtureLockStrategy.java
similarity index 59%
copy from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
copy to shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/fixture/FixtureLockStrategy.java
index 0c9cec2..92267df 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/lock/LockStrategy.java
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/java/org/apache/shardingsphere/infra/lock/fixture/FixtureLockStrategy.java
@@ -15,30 +15,29 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.lock;
+package org.apache.shardingsphere.infra.lock.fixture;
 
-/**
- * Lock strategy.
- */
-public interface LockStrategy {
+import org.apache.shardingsphere.infra.lock.LockStrategy;
+import org.apache.shardingsphere.infra.lock.LockStrategyType;
+
+public final class FixtureLockStrategy implements LockStrategy {
+    @Override
+    public boolean tryLock(final Long timeout) {
+        return false;
+    }
     
-    /**
-     * Try to get lock.
-     * 
-     * @param timeout the maximum time in milliseconds to acquire lock
-     * @return true if get the lock, false if not
-     */
-    boolean tryLock(Long timeout);
+    @Override
+    public void releaseLock() {
+        
+    }
     
-    /**
-     * Release lock.
-     */
-    void releaseLock();
+    @Override
+    public boolean checkLock() {
+        return false;
+    }
     
-    /**
-     * Check lock state.
-     * 
-     * @return true if all instances were locked, else false
-     */
-    boolean checkLock();
+    @Override
+    public String getType() {
+        return LockStrategyType.STANDARD.name();
+    }
 }
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
new file mode 100644
index 0000000..e150e66
--- /dev/null
+++ b/shardingsphere-infra/shardingsphere-infra-common/src/test/resources/META-INF/services/org.apache.shardingsphere.infra.lock.LockStrategy
@@ -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.infra.lock.fixture.FixtureLockStrategy
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
index 408e363..ee98e26 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/GovernanceBootstrapInitializer.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.proxy.initializer.impl;
 import org.apache.shardingsphere.governance.context.metadata.GovernanceMetaDataContexts;
 import org.apache.shardingsphere.governance.context.transaction.GovernanceTransactionContexts;
 import org.apache.shardingsphere.governance.core.facade.GovernanceFacade;
-import org.apache.shardingsphere.governance.core.lock.GovernanceLockStrategy;
 import org.apache.shardingsphere.governance.core.yaml.swapper.GovernanceConfigurationYamlSwapper;
 import org.apache.shardingsphere.infra.auth.builtin.DefaultAuthentication;
 import org.apache.shardingsphere.infra.auth.builtin.yaml.config.YamlAuthenticationConfiguration;
@@ -30,6 +29,7 @@ import org.apache.shardingsphere.infra.config.datasource.DataSourceConfiguration
 import org.apache.shardingsphere.infra.config.datasource.DataSourceParameter;
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.lock.LockContext;
+import org.apache.shardingsphere.infra.lock.LockStrategyType;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.ProxyConfigurationUpdater;
@@ -126,6 +126,6 @@ public final class GovernanceBootstrapInitializer extends AbstractBootstrapIniti
     
     @Override
     protected void initLockContext() {
-        LockContext.init(new GovernanceLockStrategy(governanceFacade));
+        LockContext.init(LockStrategyType.GOVERNANCE);
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/StandardBootstrapInitializer.java b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/StandardBootstrapInitializer.java
index 7e0064a..64dc027 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/StandardBootstrapInitializer.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-bootstrap/src/main/java/org/apache/shardingsphere/proxy/initializer/impl/StandardBootstrapInitializer.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.proxy.initializer.impl;
 
 import org.apache.shardingsphere.infra.context.metadata.MetaDataContexts;
 import org.apache.shardingsphere.infra.lock.LockContext;
-import org.apache.shardingsphere.infra.lock.StandardLockStrategy;
+import org.apache.shardingsphere.infra.lock.LockStrategyType;
 import org.apache.shardingsphere.proxy.config.ProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.YamlProxyConfiguration;
 import org.apache.shardingsphere.proxy.config.yaml.swapper.YamlProxyConfigurationSwapper;
@@ -47,6 +47,6 @@ public final class StandardBootstrapInitializer extends AbstractBootstrapInitial
     
     @Override
     protected void initLockContext() {
-        LockContext.init(new StandardLockStrategy());
+        LockContext.init(LockStrategyType.STANDARD);
     }
 }