From 61ab971f3a64853a384b8896e8ca29f317ac4b43 Mon Sep 17 00:00:00 2001 From: zhangliang Date: Sat, 21 Dec 2024 22:00:52 +0800 Subject: [PATCH 1/2] Add StandaloneLockContext and ClusterLockContext --- .../collect/StatisticsCollectJob.java | 4 +- ...ShardingSphereStatisticsRefreshEngine.java | 12 ++-- ...dingSphereStatisticsRefreshEngineTest.java | 8 +-- .../cluster/ClusterContextManagerBuilder.java | 4 +- .../type/ListenerAssistedSubscriber.java | 4 +- .../type/MetaDataChangedSubscriber.java | 4 +- .../cluster/lock/ClusterLockContext.java | 42 +++++++++++++ .../cluster/lock/ClusterLockContextTest.java | 59 +++++++++++++++++++ .../StandaloneContextManagerBuilder.java | 4 +- .../lock/StandaloneLockContext.java | 36 +++++++++++ 10 files changed, 157 insertions(+), 20 deletions(-) create mode 100644 mode/type/cluster/core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContext.java create mode 100644 mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContextTest.java create mode 100644 mode/type/standalone/core/src/main/java/org/apache/shardingsphere/mode/manager/standalone/lock/StandaloneLockContext.java 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 d24a581797ff6..bd5537db7201f 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 @@ 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 28457721b475c..c87b8e1904379 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.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.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 void refresh() { 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 @@ private void collectAndRefresh() { } 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 eb3d204419c89..ffc7a4e17d082 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 @@ 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.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 @@ void assertRefresh() { 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 b64d4b56d2420..becc43ad29f63 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.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.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 ContextManager build(final ContextManagerBuilderParameter param, final Ev 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 f97d84c338316..0f09b38c08662 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.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 synchronized void renew(final DropDatabaseListenerAssistedEvent event) { 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 a688874202a53..a3cc283ef21fb 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.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 synchronized void renew(final ViewDroppedEvent event) { 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 0000000000000..533642f862d21 --- /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 { + + private final LockPersistService 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 0000000000000..f90288b1a3d29 --- /dev/null +++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/lock/ClusterLockContextTest.java @@ -0,0 +1,59 @@ +/* + * 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.GlobalLockContext; +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.*; +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 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); + } +} \ No newline at end of file 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 335256ca580d2..d7f51278f9a78 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.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 ContextManager build(final ContextManagerBuilderParameter param, final Ev 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 0000000000000..f790342744e1b --- /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 { + + @Override + public boolean tryLock(final LockDefinition lockDefinition, final long timeoutMillis) { + return false; + } + + @Override + public void unlock(final LockDefinition lockDefinition) { + } +} From e3d011b5eefacad189df2bdc111ff99396bc78ef Mon Sep 17 00:00:00 2001 From: zhangliang Date: Sat, 21 Dec 2024 22:03:35 +0800 Subject: [PATCH 2/2] Add StandaloneLockContext and ClusterLockContext --- .../mode/manager/cluster/lock/ClusterLockContextTest.java | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) 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 index f90288b1a3d29..e9c5a39d9a0fd 100644 --- 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 @@ -18,7 +18,6 @@ package org.apache.shardingsphere.mode.manager.cluster.lock; import org.apache.shardingsphere.mode.lock.LockPersistService; -import org.apache.shardingsphere.mode.lock.global.GlobalLockContext; import org.apache.shardingsphere.mode.lock.global.GlobalLockDefinition; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -26,7 +25,7 @@ import org.mockito.Mock; import org.mockito.junit.jupiter.MockitoExtension; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; @@ -56,4 +55,4 @@ void assertUnlock() { lockContext.unlock(lockDefinition); verify(lockPersistService).unlock(lockDefinition); } -} \ No newline at end of file +}