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 2f7dfcd711e Add StandaloneLockContext and ClusterLockContext (#34115)
2f7dfcd711e is described below

commit 2f7dfcd711e0db06fac5d94472bd692973535c6e
Author: Liang Zhang <[email protected]>
AuthorDate: Sat Dec 21 22:10:26 2024 +0800

    Add StandaloneLockContext and ClusterLockContext (#34115)
    
    * Add StandaloneLockContext and ClusterLockContext
    
    * Add StandaloneLockContext and ClusterLockContext
---
 .../statistics/collect/StatisticsCollectJob.java   |  4 +-
 .../ShardingSphereStatisticsRefreshEngine.java     | 12 ++---
 .../ShardingSphereStatisticsRefreshEngineTest.java |  8 +--
 .../cluster/ClusterContextManagerBuilder.java      |  4 +-
 .../type/ListenerAssistedSubscriber.java           |  4 +-
 .../subscriber/type/MetaDataChangedSubscriber.java |  4 +-
 .../manager/cluster/lock/ClusterLockContext.java   | 42 ++++++++++++++++
 .../cluster/lock/ClusterLockContextTest.java       | 58 ++++++++++++++++++++++
 .../StandaloneContextManagerBuilder.java           |  4 +-
 .../standalone/lock/StandaloneLockContext.java     | 36 ++++++++++++++
 10 files changed, 156 insertions(+), 20 deletions(-)

diff --git 
a/kernel/schedule/core/src/main/java/org/apache/shardingsphere/schedule/core/job/statistics/collect/StatisticsCollectJob.java
 
b/kernel/schedule/core/src/main/java/org/apache/shardingsphere/schedule/core/job/statistics/collect/StatisticsCollectJob.java
index d24a581797f..bd5537db720 100644
--- 
a/kernel/schedule/core/src/main/java/org/apache/shardingsphere/schedule/core/job/statistics/collect/StatisticsCollectJob.java
+++ 
b/kernel/schedule/core/src/main/java/org/apache/shardingsphere/schedule/core/job/statistics/collect/StatisticsCollectJob.java
@@ -20,8 +20,8 @@ package 
org.apache.shardingsphere.schedule.core.job.statistics.collect;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.elasticjob.api.ShardingContext;
 import org.apache.shardingsphere.elasticjob.simple.job.SimpleJob;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
+import org.apache.shardingsphere.mode.manager.cluster.lock.ClusterLockContext;
 import 
org.apache.shardingsphere.mode.manager.cluster.persist.service.GlobalLockPersistService;
 import 
org.apache.shardingsphere.mode.metadata.refresher.ShardingSphereStatisticsRefreshEngine;
 import 
org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
@@ -39,7 +39,7 @@ public final class StatisticsCollectJob implements SimpleJob {
     public void execute(final ShardingContext shardingContext) {
         PersistRepository repository = 
contextManager.getPersistServiceFacade().getRepository();
         if (repository instanceof ClusterPersistRepository) {
-            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
GlobalLockContext(new GlobalLockPersistService((ClusterPersistRepository) 
repository))).refresh();
+            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
ClusterLockContext(new GlobalLockPersistService((ClusterPersistRepository) 
repository))).refresh();
         }
     }
 }
diff --git 
a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngine.java
 
b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngine.java
index 28457721b47..c87b8e19043 100644
--- 
a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngine.java
+++ 
b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngine.java
@@ -21,7 +21,7 @@ import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
 import 
org.apache.shardingsphere.infra.config.props.temporary.TemporaryConfigurationPropertyKey;
 import 
org.apache.shardingsphere.infra.executor.kernel.thread.ExecutorThreadFactoryBuilder;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockNames;
+import org.apache.shardingsphere.infra.lock.LockContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereSchema;
@@ -34,10 +34,10 @@ import 
org.apache.shardingsphere.infra.metadata.statistics.ShardingSphereTableDa
 import 
org.apache.shardingsphere.infra.metadata.statistics.collector.ShardingSphereStatisticsCollector;
 import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
 import 
org.apache.shardingsphere.infra.yaml.data.swapper.YamlShardingSphereRowDataSwapper;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
+import 
org.apache.shardingsphere.metadata.persist.data.AlteredShardingSphereDatabaseData;
 import org.apache.shardingsphere.mode.lock.global.GlobalLockDefinition;
+import org.apache.shardingsphere.mode.lock.global.GlobalLockNames;
 import org.apache.shardingsphere.mode.manager.ContextManager;
-import 
org.apache.shardingsphere.metadata.persist.data.AlteredShardingSphereDatabaseData;
 
 import java.util.ArrayList;
 import java.util.Map;
@@ -59,7 +59,7 @@ public final class ShardingSphereStatisticsRefreshEngine {
     
     private final ContextManager contextManager;
     
-    private final GlobalLockContext globalLockContext;
+    private final LockContext lockContext;
     
     /**
      * Async refresh.
@@ -85,7 +85,7 @@ public final class ShardingSphereStatisticsRefreshEngine {
     
     private void collectAndRefresh() {
         GlobalLockDefinition lockDefinition = new 
GlobalLockDefinition(GlobalLockNames.STATISTICS.getLockName());
-        if (globalLockContext.tryLock(lockDefinition, 5000L)) {
+        if (lockContext.tryLock(lockDefinition, 5000L)) {
             try {
                 ShardingSphereStatistics statistics = 
contextManager.getMetaDataContexts().getStatistics();
                 ShardingSphereMetaData metaData = 
contextManager.getMetaDataContexts().getMetaData();
@@ -97,7 +97,7 @@ public final class ShardingSphereStatisticsRefreshEngine {
                 }
                 compareAndUpdate(changedStatistics);
             } finally {
-                globalLockContext.unlock(lockDefinition);
+                lockContext.unlock(lockDefinition);
             }
         }
     }
diff --git 
a/mode/core/src/test/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngineTest.java
 
b/mode/core/src/test/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngineTest.java
index eb3d204419c..ffc7a4e17d0 100644
--- 
a/mode/core/src/test/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngineTest.java
+++ 
b/mode/core/src/test/java/org/apache/shardingsphere/mode/metadata/refresher/ShardingSphereStatisticsRefreshEngineTest.java
@@ -20,6 +20,7 @@ package org.apache.shardingsphere.mode.metadata.refresher;
 import org.apache.shardingsphere.infra.config.props.ConfigurationProperties;
 import 
org.apache.shardingsphere.infra.config.props.temporary.TemporaryConfigurationProperties;
 import 
org.apache.shardingsphere.infra.config.props.temporary.TemporaryConfigurationPropertyKey;
+import org.apache.shardingsphere.infra.lock.LockContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import 
org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import 
org.apache.shardingsphere.infra.metadata.database.schema.model.ShardingSphereColumn;
@@ -29,7 +30,6 @@ import 
org.apache.shardingsphere.infra.metadata.statistics.ShardingSphereDatabas
 import 
org.apache.shardingsphere.infra.metadata.statistics.ShardingSphereSchemaData;
 import 
org.apache.shardingsphere.infra.metadata.statistics.ShardingSphereStatistics;
 import 
org.apache.shardingsphere.infra.metadata.statistics.ShardingSphereTableData;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.test.util.PropertiesBuilder;
 import org.apache.shardingsphere.test.util.PropertiesBuilder.Property;
@@ -59,9 +59,9 @@ class ShardingSphereStatisticsRefreshEngineTest {
         
when(contextManager.getMetaDataContexts().getMetaData().getProps()).thenReturn(new
 ConfigurationProperties(new Properties()));
         
when(contextManager.getMetaDataContexts().getMetaData().getTemporaryProps()).thenReturn(new
 TemporaryConfigurationProperties(
                 PropertiesBuilder.build(new 
Property(TemporaryConfigurationPropertyKey.PROXY_META_DATA_COLLECTOR_ENABLED.getKey(),
 Boolean.TRUE.toString()))));
-        GlobalLockContext globalLockContext = mock(GlobalLockContext.class);
-        when(globalLockContext.tryLock(any(), anyLong())).thenReturn(true);
-        new ShardingSphereStatisticsRefreshEngine(contextManager, 
globalLockContext).refresh();
+        LockContext lockContext = mock(LockContext.class);
+        when(lockContext.tryLock(any(), anyLong())).thenReturn(true);
+        new ShardingSphereStatisticsRefreshEngine(contextManager, 
lockContext).refresh();
         
verify(contextManager.getPersistServiceFacade().getMetaDataPersistService().getShardingSphereDataPersistService()).update(any());
     }
     
diff --git 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
index b64d4b56d24..becc43ad29f 100644
--- 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
+++ 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/ClusterContextManagerBuilder.java
@@ -29,7 +29,6 @@ import 
org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventSubscriber;
 import org.apache.shardingsphere.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.event.deliver.DeliverEventSubscriber;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilder;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
@@ -37,6 +36,7 @@ import 
org.apache.shardingsphere.mode.manager.cluster.event.ClusterEventSubscrib
 import 
org.apache.shardingsphere.mode.manager.cluster.event.dispatch.listener.DataChangedEventListenerRegistry;
 import 
org.apache.shardingsphere.mode.manager.cluster.event.dispatch.subscriber.ClusterDispatchEventSubscriberRegistry;
 import 
org.apache.shardingsphere.mode.manager.cluster.exception.MissingRequiredClusterRepositoryConfigurationException;
+import org.apache.shardingsphere.mode.manager.cluster.lock.ClusterLockContext;
 import 
org.apache.shardingsphere.mode.manager.cluster.persist.service.GlobalLockPersistService;
 import 
org.apache.shardingsphere.mode.manager.cluster.workerid.ClusterWorkerIdGenerator;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
@@ -59,7 +59,7 @@ public final class ClusterContextManagerBuilder implements 
ContextManagerBuilder
         ClusterPersistRepositoryConfiguration config = 
(ClusterPersistRepositoryConfiguration) modeConfig.getRepository();
         ComputeNodeInstanceContext computeNodeInstanceContext = new 
ComputeNodeInstanceContext(new ComputeNodeInstance(param.getInstanceMetaData(), 
param.getLabels()), modeConfig, eventBusContext);
         ClusterPersistRepository repository = 
getClusterPersistRepository(config, computeNodeInstanceContext);
-        LockContext<?> lockContext = new GlobalLockContext(new 
GlobalLockPersistService(repository));
+        LockContext<?> lockContext = new ClusterLockContext(new 
GlobalLockPersistService(repository));
         computeNodeInstanceContext.init(new 
ClusterWorkerIdGenerator(repository, param.getInstanceMetaData().getId()), 
lockContext);
         MetaDataPersistService metaDataPersistService = new 
MetaDataPersistService(repository);
         MetaDataContexts metaDataContexts = 
MetaDataContextsFactory.create(metaDataPersistService, param, 
computeNodeInstanceContext);
diff --git 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/ListenerAssistedSubscriber.java
 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/ListenerAssistedSubscriber.java
index f97d84c3383..0f09b38c086 100644
--- 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/ListenerAssistedSubscriber.java
+++ 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/ListenerAssistedSubscriber.java
@@ -23,10 +23,10 @@ import 
org.apache.shardingsphere.infra.instance.metadata.InstanceType;
 import org.apache.shardingsphere.metadata.persist.node.DatabaseMetaDataNode;
 import 
org.apache.shardingsphere.mode.event.dispatch.assisted.CreateDatabaseListenerAssistedEvent;
 import 
org.apache.shardingsphere.mode.event.dispatch.assisted.DropDatabaseListenerAssistedEvent;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import 
org.apache.shardingsphere.mode.manager.cluster.event.dispatch.listener.type.DatabaseMetaDataChangedListener;
 import 
org.apache.shardingsphere.mode.manager.cluster.event.dispatch.subscriber.DispatchEventSubscriber;
+import org.apache.shardingsphere.mode.manager.cluster.lock.ClusterLockContext;
 import 
org.apache.shardingsphere.mode.manager.cluster.persist.service.GlobalLockPersistService;
 import 
org.apache.shardingsphere.mode.metadata.refresher.ShardingSphereStatisticsRefreshEngine;
 import 
org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
@@ -75,7 +75,7 @@ public final class ListenerAssistedSubscriber implements 
DispatchEventSubscriber
     private void refreshStatisticsData() {
         if 
(contextManager.getComputeNodeInstanceContext().getModeConfiguration().isCluster()
                 && InstanceType.PROXY == 
contextManager.getComputeNodeInstanceContext().getInstance().getMetaData().getType())
 {
-            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
GlobalLockContext(new GlobalLockPersistService(repository))).asyncRefresh();
+            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
ClusterLockContext(new GlobalLockPersistService(repository))).asyncRefresh();
         }
     }
 }
diff --git 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/MetaDataChangedSubscriber.java
 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/MetaDataChangedSubscriber.java
index a688874202a..a3cc283ef21 100644
--- 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/MetaDataChangedSubscriber.java
+++ 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/event/dispatch/subscriber/type/MetaDataChangedSubscriber.java
@@ -28,9 +28,9 @@ import 
org.apache.shardingsphere.mode.event.dispatch.metadata.schema.table.Table
 import 
org.apache.shardingsphere.mode.event.dispatch.metadata.schema.table.TableDroppedEvent;
 import 
org.apache.shardingsphere.mode.event.dispatch.metadata.schema.view.ViewCreatedOrAlteredEvent;
 import 
org.apache.shardingsphere.mode.event.dispatch.metadata.schema.view.ViewDroppedEvent;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import 
org.apache.shardingsphere.mode.manager.cluster.event.dispatch.subscriber.DispatchEventSubscriber;
+import org.apache.shardingsphere.mode.manager.cluster.lock.ClusterLockContext;
 import 
org.apache.shardingsphere.mode.manager.cluster.persist.service.GlobalLockPersistService;
 import 
org.apache.shardingsphere.mode.metadata.refresher.ShardingSphereStatisticsRefreshEngine;
 import 
org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
@@ -128,7 +128,7 @@ public final class MetaDataChangedSubscriber implements 
DispatchEventSubscriber
     private void refreshStatisticsData() {
         if 
(contextManager.getComputeNodeInstanceContext().getModeConfiguration().isCluster()
                 && InstanceType.PROXY == 
contextManager.getComputeNodeInstanceContext().getInstance().getMetaData().getType())
 {
-            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
GlobalLockContext(new GlobalLockPersistService(repository))).asyncRefresh();
+            new ShardingSphereStatisticsRefreshEngine(contextManager, new 
ClusterLockContext(new GlobalLockPersistService(repository))).asyncRefresh();
         }
     }
 }
diff --git 
a/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContext.java
 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContext.java
new file mode 100644
index 00000000000..533642f862d
--- /dev/null
+++ 
b/mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContext.java
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.manager.cluster.lock;
+
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.lock.LockContext;
+import org.apache.shardingsphere.mode.lock.LockPersistService;
+import org.apache.shardingsphere.mode.lock.global.GlobalLockDefinition;
+
+/**
+ * Cluster global lock context.
+ */
+@RequiredArgsConstructor
+public final class ClusterLockContext implements 
LockContext<GlobalLockDefinition> {
+    
+    private final LockPersistService<GlobalLockDefinition> 
globalLockPersistService;
+    
+    @Override
+    public boolean tryLock(final GlobalLockDefinition lockDefinition, final 
long timeoutMillis) {
+        return globalLockPersistService.tryLock(lockDefinition, timeoutMillis);
+    }
+    
+    @Override
+    public void unlock(final GlobalLockDefinition lockDefinition) {
+        globalLockPersistService.unlock(lockDefinition);
+    }
+}
diff --git 
a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContextTest.java
 
b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContextTest.java
new file mode 100644
index 00000000000..e9c5a39d9a0
--- /dev/null
+++ 
b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContextTest.java
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.manager.cluster.lock;
+
+import org.apache.shardingsphere.mode.lock.LockPersistService;
+import org.apache.shardingsphere.mode.lock.global.GlobalLockDefinition;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+import org.mockito.Mock;
+import org.mockito.junit.jupiter.MockitoExtension;
+
+import static org.junit.jupiter.api.Assertions.assertTrue;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+@ExtendWith(MockitoExtension.class)
+class ClusterLockContextTest {
+    
+    private final GlobalLockDefinition lockDefinition = new 
GlobalLockDefinition("foo_lock");
+    
+    @Mock
+    private LockPersistService<GlobalLockDefinition> lockPersistService;
+    
+    private ClusterLockContext lockContext;
+    
+    @BeforeEach
+    void init() {
+        lockContext = new ClusterLockContext(lockPersistService);
+    }
+    
+    @Test
+    void assertTryLock() {
+        when(lockPersistService.tryLock(lockDefinition, 
3000L)).thenReturn(true);
+        assertTrue(lockContext.tryLock(lockDefinition, 3000L));
+    }
+    
+    @Test
+    void assertUnlock() {
+        lockContext.unlock(lockDefinition);
+        verify(lockPersistService).unlock(lockDefinition);
+    }
+}
diff --git 
a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
 
b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
index 335256ca580..d7f51278f9a 100644
--- 
a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
+++ 
b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/StandaloneContextManagerBuilder.java
@@ -23,10 +23,10 @@ import 
org.apache.shardingsphere.infra.instance.ComputeNodeInstanceContext;
 import org.apache.shardingsphere.infra.spi.type.typed.TypedSPILoader;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.metadata.persist.MetaDataPersistService;
-import org.apache.shardingsphere.mode.lock.global.GlobalLockContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilder;
 import org.apache.shardingsphere.mode.manager.ContextManagerBuilderParameter;
+import 
org.apache.shardingsphere.mode.manager.standalone.lock.StandaloneLockContext;
 import 
org.apache.shardingsphere.mode.manager.standalone.workerid.StandaloneWorkerIdGenerator;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.MetaDataContextsFactory;
@@ -47,7 +47,7 @@ public final class StandaloneContextManagerBuilder implements 
ContextManagerBuil
                 StandalonePersistRepository.class, null == repositoryConfig ? 
null : repositoryConfig.getType(), null == repositoryConfig ? new Properties() 
: repositoryConfig.getProps());
         MetaDataPersistService persistService = new 
MetaDataPersistService(repository);
         ComputeNodeInstanceContext computeNodeInstanceContext = new 
ComputeNodeInstanceContext(new 
ComputeNodeInstance(param.getInstanceMetaData()), param.getModeConfiguration(), 
eventBusContext);
-        computeNodeInstanceContext.init(new StandaloneWorkerIdGenerator(), new 
GlobalLockContext(null));
+        computeNodeInstanceContext.init(new StandaloneWorkerIdGenerator(), new 
StandaloneLockContext());
         MetaDataContexts metaDataContexts = 
MetaDataContextsFactory.create(persistService, param, 
computeNodeInstanceContext);
         return new ContextManager(metaDataContexts, 
computeNodeInstanceContext, repository);
     }
diff --git 
a/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/lock/StandaloneLockContext.java
 
b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/lock/StandaloneLockContext.java
new file mode 100644
index 00000000000..f790342744e
--- /dev/null
+++ 
b/mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/lock/StandaloneLockContext.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.mode.manager.standalone.lock;
+
+import org.apache.shardingsphere.infra.lock.LockContext;
+import org.apache.shardingsphere.infra.lock.LockDefinition;
+
+/**
+ * Standalone global lock context.
+ */
+public final class StandaloneLockContext implements 
LockContext<LockDefinition> {
+    
+    @Override
+    public boolean tryLock(final LockDefinition lockDefinition, final long 
timeoutMillis) {
+        return false;
+    }
+    
+    @Override
+    public void unlock(final LockDefinition lockDefinition) {
+    }
+}

Reply via email to