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) {
+ }
+}