You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by su...@apache.org on 2022/10/10 08:20:51 UTC

[shardingsphere] branch master updated: Decouple mode and schedule module dependency (#21445)

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

sunnianjun 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 9e3d4e6399c Decouple mode and schedule module dependency (#21445)
9e3d4e6399c is described below

commit 9e3d4e6399cb8adf39afc99d4a5360db963ee770
Author: zhaojinchao <zh...@apache.org>
AuthorDate: Mon Oct 10 16:20:42 2022 +0800

    Decouple mode and schedule module dependency (#21445)
    
    * Decouple mode and schedule dependency
    
    * Fix checkstyle
---
 .../collector/ProxyInfoCollectorTest.java          |  3 +--
 .../service/PrometheusPluginBootServiceTest.java   |  3 +--
 features/db-discovery/core/pom.xml                 |  5 ++++
 .../dbdiscovery/rule/DatabaseDiscoveryRule.java    | 10 ++++++--
 .../keygen/SnowflakeKeyGenerateAlgorithmTest.java  |  5 ++--
 .../checker/ShardingRouteCacheableCheckerTest.java |  2 +-
 .../CosIdSnowflakeKeyGenerateAlgorithmTest.java    | 11 ++++----
 .../infra/instance/InstanceContext.java            |  7 +-----
 .../infra/instance/InstanceContextTest.java        | 21 ++++++++--------
 .../builder/global/GlobalRulesBuilderTest.java     |  3 +--
 .../fixture/FixtureClusterPersistRepository.java   |  2 +-
 mode/core/pom.xml                                  |  5 ----
 .../schema/TableMetaDataPersistService.java        |  4 +--
 .../service/schema/ViewMetaDataPersistService.java |  4 +--
 .../cluster/ClusterContextManagerBuilder.java      |  4 +--
 .../fixture/ClusterPersistRepositoryFixture.java   |  2 +-
 ...ProcessListClusterPersistRepositoryFixture.java |  2 +-
 .../cluster/ClusterPersistRepository.java          |  1 -
 .../cluster/LeaderExecutionCallback.java           | 29 ++++++++++++++++++++++
 .../cluster/repository/provider/consul/pom.xml     |  4 +++
 .../cluster/consul/ConsulRepository.java           |  2 +-
 .../consul/lock/ConsulInternalLockProvider.java    |  6 ++---
 .../repository/cluster/etcd/EtcdRepository.java    |  2 +-
 .../repository/cluster/nacos/NacosRepository.java  |  2 +-
 .../zookeeper/CuratorZookeeperRepository.java      |  2 +-
 .../StandaloneContextManagerBuilder.java           |  4 +--
 .../standalone/jdbc/JDBCRepositoryTest.java        |  3 +--
 .../fixture/ClusterPersistRepositoryFixture.java   |  2 +-
 .../fixture/TestClusterPersistRepository.java      |  2 +-
 29 files changed, 87 insertions(+), 65 deletions(-)

diff --git a/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java b/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
index 591f070d1c3..0a1b09906e5 100644
--- a/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
+++ b/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/collector/ProxyInfoCollectorTest.java
@@ -25,7 +25,6 @@ import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.lock.LockContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.data.ShardingSphereData;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.standalone.workerid.generator.StandaloneWorkerIdGenerator;
@@ -44,7 +43,7 @@ public final class ProxyInfoCollectorTest extends ProxyContextRestorer {
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), new ShardingSphereMetaData(), new ShardingSphereData());
         InstanceContext instanceContext = new InstanceContext(
                 new ComputeNodeInstance(mock(InstanceMetaData.class)), new StandaloneWorkerIdGenerator(), new ModeConfiguration("Standalone", null), mock(LockContext.class),
-                new EventBusContext(), mock(ScheduleContext.class));
+                new EventBusContext());
         ProxyContext.init(new ContextManager(metaDataContexts, instanceContext));
         assertFalse(new ProxyInfoCollector().collect().isEmpty());
     }
diff --git a/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java b/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
index 19ee99da4fd..ac571bf4854 100644
--- a/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
+++ b/agent/plugins/metrics/prometheus/src/test/java/org/apache/shardingsphere/agent/metrics/prometheus/service/PrometheusPluginBootServiceTest.java
@@ -26,7 +26,6 @@ import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.lock.LockContext;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.data.ShardingSphereData;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.standalone.workerid.generator.StandaloneWorkerIdGenerator;
@@ -57,7 +56,7 @@ public final class PrometheusPluginBootServiceTest extends ProxyContextRestorer
         MetaDataContexts metaDataContexts = new MetaDataContexts(mock(MetaDataPersistService.class), new ShardingSphereMetaData(), new ShardingSphereData());
         InstanceContext instanceContext = new InstanceContext(
                 new ComputeNodeInstance(mock(InstanceMetaData.class)), new StandaloneWorkerIdGenerator(), new ModeConfiguration("Standalone", null), mock(LockContext.class),
-                new EventBusContext(), mock(ScheduleContext.class));
+                new EventBusContext());
         ProxyContext.init(new ContextManager(metaDataContexts, instanceContext));
         PROMETHEUS_PLUGIN_BOOT_SERVICE.start(new PluginConfiguration("localhost", 8090, "", createProperties()));
         new Socket().connect(new InetSocketAddress("localhost", 8090));
diff --git a/features/db-discovery/core/pom.xml b/features/db-discovery/core/pom.xml
index 2e2bae999a4..25c8415d1f6 100644
--- a/features/db-discovery/core/pom.xml
+++ b/features/db-discovery/core/pom.xml
@@ -53,6 +53,11 @@
             <artifactId>shardingsphere-mode-core</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-schedule-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
diff --git a/features/db-discovery/core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java b/features/db-discovery/core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
index 5ff47b7284f..d4ec73ad938 100644
--- a/features/db-discovery/core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
+++ b/features/db-discovery/core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
@@ -39,9 +39,11 @@ import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedR
 import org.apache.shardingsphere.infra.rule.identifier.type.DynamicDataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.exportable.ExportableRule;
 import org.apache.shardingsphere.infra.schedule.CronJob;
+import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
 import org.apache.shardingsphere.mode.metadata.storage.event.StorageNodeDataSourceChangedEvent;
+import org.apache.shardingsphere.schedule.core.ScheduleContextFactory;
 
 import javax.sql.DataSource;
 import java.util.Collection;
@@ -72,11 +74,14 @@ public final class DatabaseDiscoveryRule implements DatabaseRule, DataSourceCont
     
     private final InstanceContext instanceContext;
     
+    private final ScheduleContext scheduleContext;
+    
     public DatabaseDiscoveryRule(final String databaseName, final Map<String, DataSource> dataSourceMap, final DatabaseDiscoveryRuleConfiguration ruleConfig, final InstanceContext instanceContext) {
         configuration = ruleConfig;
         this.databaseName = databaseName;
         this.dataSourceMap = dataSourceMap;
         this.instanceContext = instanceContext;
+        this.scheduleContext = ScheduleContextFactory.newInstance(instanceContext.getModeConfiguration());
         discoveryTypes = getDiscoveryProviderAlgorithms(ruleConfig.getDiscoveryTypes());
         dataSourceRules = getDataSourceRules(ruleConfig.getDataSources(), ruleConfig.getDiscoveryHeartbeats());
         findPrimaryReplicaRelationship(databaseName, dataSourceMap);
@@ -89,6 +94,7 @@ public final class DatabaseDiscoveryRule implements DatabaseRule, DataSourceCont
         this.databaseName = databaseName;
         this.dataSourceMap = dataSourceMap;
         this.instanceContext = instanceContext;
+        this.scheduleContext = ScheduleContextFactory.newInstance(instanceContext.getModeConfiguration());
         discoveryTypes = ruleConfig.getDiscoveryTypes();
         dataSourceRules = getDataSourceRules(ruleConfig.getDataSources(), ruleConfig.getDiscoveryHeartbeats());
         findPrimaryReplicaRelationship(databaseName, dataSourceMap);
@@ -167,7 +173,7 @@ public final class DatabaseDiscoveryRule implements DatabaseRule, DataSourceCont
     public void closeHeartBeatJob() {
         for (Entry<String, DatabaseDiscoveryDataSourceRule> entry : dataSourceRules.entrySet()) {
             DatabaseDiscoveryDataSourceRule rule = entry.getValue();
-            instanceContext.getScheduleContext().closeSchedule(rule.getDatabaseDiscoveryProviderAlgorithm().getType() + "-" + databaseName + "-" + rule.getGroupName());
+            scheduleContext.closeSchedule(rule.getDatabaseDiscoveryProviderAlgorithm().getType() + "-" + databaseName + "-" + rule.getGroupName());
         }
     }
     
@@ -178,7 +184,7 @@ public final class DatabaseDiscoveryRule implements DatabaseRule, DataSourceCont
             CronJob job = new CronJob(jobName, each -> new HeartbeatJob(databaseName, rule.getGroupName(), rule.getPrimaryDataSourceName(), rule.getDataSourceGroup(dataSourceMap),
                     rule.getDatabaseDiscoveryProviderAlgorithm(), rule.getDisabledDataSourceNames(), instanceContext.getEventBusContext()).execute(null),
                     rule.getHeartbeatProps().getProperty("keep-alive-cron"));
-            instanceContext.getScheduleContext().startSchedule(job);
+            scheduleContext.startSchedule(job);
         }
     }
     
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
index b92d249b7c1..bf01cc8004d 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/algorithm/keygen/SnowflakeKeyGenerateAlgorithmTest.java
@@ -21,7 +21,6 @@ import lombok.SneakyThrows;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.instance.InstanceContextAware;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
@@ -208,7 +207,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     public void assertSetWorkerIdFailureWhenNegative() {
         SnowflakeKeyGenerateAlgorithm algorithm = (SnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("SNOWFLAKE", new Properties()));
         InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(-1L),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext(), mock(ScheduleContext.class));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext());
         algorithm.setInstanceContext(instanceContext);
         algorithm.generateKey();
     }
@@ -224,7 +223,7 @@ public final class SnowflakeKeyGenerateAlgorithmTest {
     public void assertSetWorkerIdFailureWhenOutOfRange() {
         SnowflakeKeyGenerateAlgorithm algorithm = (SnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("SNOWFLAKE", new Properties()));
         InstanceContext instanceContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext(), mock(ScheduleContext.class));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), new EventBusContext());
         algorithm.setInstanceContext(instanceContext);
         algorithm.generateKey();
     }
diff --git a/features/sharding/plugin/cache/src/test/java/org/apache/shardingsphere/sharding/cache/checker/ShardingRouteCacheableCheckerTest.java b/features/sharding/plugin/cache/src/test/java/org/apache/shardingsphere/sharding/cache/checker/ShardingRouteCacheableCheckerTest.java
index a4a99e12671..7d8906a75fa 100644
--- a/features/sharding/plugin/cache/src/test/java/org/apache/shardingsphere/sharding/cache/checker/ShardingRouteCacheableCheckerTest.java
+++ b/features/sharding/plugin/cache/src/test/java/org/apache/shardingsphere/sharding/cache/checker/ShardingRouteCacheableCheckerTest.java
@@ -140,7 +140,7 @@ public final class ShardingRouteCacheableCheckerTest {
         ShardingTableRuleConfiguration nonCacheableTableSharding = new ShardingTableRuleConfiguration("t_non_cacheable_table_sharding", "ds_0.t_non_cacheable_table_sharding_${0..1}");
         nonCacheableTableSharding.setTableShardingStrategy(new StandardShardingStrategyConfiguration("id", "inline"));
         configuration.getTables().add(nonCacheableTableSharding);
-        return new ShardingRule(configuration, Arrays.asList("ds_0", "ds_1"), new InstanceContext(mock(ComputeNodeInstance.class), props -> 0, null, null, null, null));
+        return new ShardingRule(configuration, Arrays.asList("ds_0", "ds_1"), new InstanceContext(mock(ComputeNodeInstance.class), props -> 0, null, null, null));
     }
     
     private ShardingCacheRule prepareShardingCacheRule(final ShardingRule shardingRule) {
diff --git a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
index ae41c79edf7..6028213c7ec 100644
--- a/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
+++ b/features/sharding/plugin/cosid/src/test/java/org/apache/shardingsphere/sharding/cosid/algorithm/keygen/CosIdSnowflakeKeyGenerateAlgorithmTest.java
@@ -28,7 +28,6 @@ import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.lock.LockContext;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.sharding.cosid.algorithm.keygen.fixture.WorkerIdGeneratorFixture;
 import org.apache.shardingsphere.sharding.factory.KeyGenerateAlgorithmFactory;
@@ -61,7 +60,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         CosIdSnowflakeKeyGenerateAlgorithm algorithm = (CosIdSnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(
                 new AlgorithmConfiguration("COSID_SNOWFLAKE", new Properties()));
         algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext, mock(ScheduleContext.class)));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext));
         long firstActualKey = (Long) algorithm.generateKey();
         long secondActualKey = (Long) algorithm.generateKey();
         SnowflakeIdState firstActualState = snowflakeIdStateParser.parse(firstActualKey);
@@ -77,7 +76,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
     public void assertGenerateKeyModUniformity() {
         CosIdSnowflakeKeyGenerateAlgorithm algorithm = (CosIdSnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(new AlgorithmConfiguration("COSID_SNOWFLAKE", new Properties()));
         algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext, mock(ScheduleContext.class)));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext));
         int divisor = 4;
         int total = 99999;
         int avg = total / divisor;
@@ -122,7 +121,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
                 new AlgorithmConfiguration("COSID_SNOWFLAKE", props));
         algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)),
                 new WorkerIdGeneratorFixture(FIXTURE_WORKER_ID), new ModeConfiguration("Standalone", null),
-                mock(LockContext.class), eventBusContext, mock(ScheduleContext.class)));
+                mock(LockContext.class), eventBusContext));
         Comparable<?> actualKey = algorithm.generateKey();
         assertThat(actualKey, instanceOf(String.class));
         String actualStringKey = (String) actualKey;
@@ -143,7 +142,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         CosIdSnowflakeKeyGenerateAlgorithm algorithm = (CosIdSnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(
                 new AlgorithmConfiguration("COSID_SNOWFLAKE", new Properties()));
         algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(-1),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext, mock(ScheduleContext.class)));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext));
         algorithm.generateKey();
     }
     
@@ -152,7 +151,7 @@ public final class CosIdSnowflakeKeyGenerateAlgorithmTest {
         CosIdSnowflakeKeyGenerateAlgorithm algorithm = (CosIdSnowflakeKeyGenerateAlgorithm) KeyGenerateAlgorithmFactory.newInstance(
                 new AlgorithmConfiguration("COSID_SNOWFLAKE", new Properties()));
         algorithm.setInstanceContext(new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(1024),
-                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext, mock(ScheduleContext.class)));
+                new ModeConfiguration("Standalone", null), mock(LockContext.class), eventBusContext));
         algorithm.generateKey();
     }
 }
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
index 0931e50f77f..e5241e2a7fa 100644
--- a/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/instance/InstanceContext.java
@@ -20,7 +20,6 @@ package org.apache.shardingsphere.infra.instance;
 import lombok.AccessLevel;
 import lombok.Getter;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceType;
@@ -51,19 +50,15 @@ public final class InstanceContext {
     
     private final EventBusContext eventBusContext;
     
-    private final ScheduleContext scheduleContext;
-    
     private final Collection<ComputeNodeInstance> allClusterInstances = new LinkedList<>();
     
     public InstanceContext(final ComputeNodeInstance instance, final WorkerIdGenerator workerIdGenerator,
-                           final ModeConfiguration modeConfiguration, final LockContext lockContext,
-                           final EventBusContext eventBusContext, final ScheduleContext scheduleContext) {
+                           final ModeConfiguration modeConfiguration, final LockContext lockContext, final EventBusContext eventBusContext) {
         this.instance = instance;
         this.workerIdGenerator = workerIdGenerator;
         this.modeConfiguration = modeConfiguration;
         this.lockContext = lockContext;
         this.eventBusContext = eventBusContext;
-        this.scheduleContext = scheduleContext;
     }
     
     /**
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
index 1f5433c7dee..04a36e310ad 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/instance/InstanceContextTest.java
@@ -18,7 +18,6 @@
 package org.apache.shardingsphere.infra.instance;
 
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.apache.shardingsphere.infra.instance.metadata.InstanceMetaData;
 import org.apache.shardingsphere.infra.instance.fixture.WorkerIdGeneratorFixture;
@@ -54,8 +53,8 @@ public final class InstanceContextTest {
     public void assertUpdateInstanceStatus() {
         InstanceMetaData instanceMetaData = mock(InstanceMetaData.class);
         when(instanceMetaData.getId()).thenReturn("foo_instance_id");
-        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
-                modeConfig, lockContext, eventBusContext, mock(ScheduleContext.class));
+        InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
+                lockContext, eventBusContext);
         StateType actual = context.getInstance().getState().getCurrentState();
         assertThat(actual, is(StateType.OK));
         context.updateInstanceStatus(instanceMetaData.getId(), Collections.singleton(StateType.CIRCUIT_BREAK.name()));
@@ -71,14 +70,14 @@ public final class InstanceContextTest {
         ComputeNodeInstance computeNodeInstance = mock(ComputeNodeInstance.class);
         when(computeNodeInstance.getWorkerId()).thenReturn(0L);
         InstanceContext context = new InstanceContext(computeNodeInstance, new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         assertThat(context.getWorkerId(), is(0L));
     }
     
     @Test
     public void assertGenerateWorkerId() {
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         long actual = context.generateWorkerId(new Properties());
         assertThat(actual, is(Long.MIN_VALUE));
     }
@@ -88,7 +87,7 @@ public final class InstanceContextTest {
         InstanceMetaData instanceMetaData = mock(InstanceMetaData.class);
         when(instanceMetaData.getId()).thenReturn("foo_instance_id");
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(instanceMetaData), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         Set<String> expected = new LinkedHashSet<>(Arrays.asList("label_1", "label_2"));
         context.updateLabel("foo_instance_id", expected);
         Collection<String> actual = context.getInstance().getLabels();
@@ -99,7 +98,7 @@ public final class InstanceContextTest {
     public void assertGetInstance() {
         ComputeNodeInstance expected = new ComputeNodeInstance(mock(InstanceMetaData.class));
         InstanceContext context = new InstanceContext(expected, new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         ComputeNodeInstance actual = context.getInstance();
         assertThat(actual, is(expected));
     }
@@ -107,7 +106,7 @@ public final class InstanceContextTest {
     @Test
     public void assertGetState() {
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         StateContext actual = context.getInstance().getState();
         assertNotNull(actual);
     }
@@ -115,7 +114,7 @@ public final class InstanceContextTest {
     @Test
     public void assertGetModeConfiguration() {
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         ModeConfiguration actual = context.getModeConfiguration();
         assertThat(actual, is(modeConfig));
     }
@@ -123,10 +122,10 @@ public final class InstanceContextTest {
     @Test
     public void assertIsCluster() {
         InstanceContext context = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE), modeConfig,
-                lockContext, eventBusContext, mock(ScheduleContext.class));
+                lockContext, eventBusContext);
         assertFalse(context.isCluster());
         InstanceContext clusterContext = new InstanceContext(new ComputeNodeInstance(mock(InstanceMetaData.class)), new WorkerIdGeneratorFixture(Long.MIN_VALUE),
-                new ModeConfiguration("Cluster", null), lockContext, eventBusContext, mock(ScheduleContext.class));
+                new ModeConfiguration("Cluster", null), lockContext, eventBusContext);
         assertTrue(clusterContext.isCluster());
     }
 }
diff --git a/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java b/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
index 7e195067caf..221135ddd0b 100644
--- a/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
+++ b/infra/common/src/test/java/org/apache/shardingsphere/infra/rule/builder/global/GlobalRulesBuilderTest.java
@@ -29,7 +29,6 @@ import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
 import org.apache.shardingsphere.infra.rule.builder.fixture.FixtureGlobalRule;
 import org.apache.shardingsphere.infra.rule.builder.fixture.FixtureGlobalRuleConfiguration;
-import org.apache.shardingsphere.infra.schedule.ScheduleContext;
 import org.apache.shardingsphere.infra.util.eventbus.EventBusContext;
 import org.junit.Test;
 
@@ -69,7 +68,7 @@ public final class GlobalRulesBuilderTest {
     private InstanceContext buildInstanceContext() {
         ComputeNodeInstance computeNodeInstance = new ComputeNodeInstance(new JDBCInstanceMetaData(UUID.randomUUID().toString()));
         ModeConfiguration modeConfig = new ModeConfiguration("Standalone", null);
-        return new InstanceContext(computeNodeInstance, createWorkerIdGenerator(), modeConfig, mock(LockContext.class), new EventBusContext(), mock(ScheduleContext.class));
+        return new InstanceContext(computeNodeInstance, createWorkerIdGenerator(), modeConfig, mock(LockContext.class), new EventBusContext());
     }
     
     private WorkerIdGenerator createWorkerIdGenerator() {
diff --git a/jdbc/spring/core/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/fixture/FixtureClusterPersistRepository.java b/jdbc/spring/core/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/fixture/FixtureClusterPersistRepository.java
index b50a40e2fd8..336b5ba356f 100644
--- a/jdbc/spring/core/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/fixture/FixtureClusterPersistRepository.java
+++ b/jdbc/spring/core/shardingsphere-jdbc-core-spring-namespace/src/test/java/org/apache/shardingsphere/spring/namespace/fixture/FixtureClusterPersistRepository.java
@@ -17,10 +17,10 @@
 
 package org.apache.shardingsphere.spring.namespace.fixture;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;
 
diff --git a/mode/core/pom.xml b/mode/core/pom.xml
index bf66e05cd38..7b3cbe33c65 100644
--- a/mode/core/pom.xml
+++ b/mode/core/pom.xml
@@ -33,11 +33,6 @@
             <artifactId>shardingsphere-infra-context</artifactId>
             <version>${project.version}</version>
         </dependency>
-        <dependency>
-            <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-schedule-core</artifactId>
-            <version>${project.version}</version>
-        </dependency>
         
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
diff --git a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/TableMetaDataPersistService.java b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/TableMetaDataPersistService.java
index d1f5a814cb5..c2cb199b75b 100644
--- a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/TableMetaDataPersistService.java
+++ b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/TableMetaDataPersistService.java
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.service.schema;
 
+import com.google.common.base.Strings;
 import lombok.RequiredArgsConstructor;
-import org.apache.commons.lang.StringUtils;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereTable;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlShardingSphereTable;
@@ -60,7 +60,7 @@ public final class TableMetaDataPersistService implements SchemaMetaDataPersistS
         Map<String, ShardingSphereTable> result = new LinkedHashMap<>(tableNames.size(), 1);
         tableNames.forEach(each -> {
             String table = repository.getDirectly(DatabaseMetaDataNode.getTableMetaDataPath(databaseName, schemaName, each));
-            if (!StringUtils.isEmpty(table)) {
+            if (!Strings.isNullOrEmpty(table)) {
                 result.put(each.toLowerCase(), new YamlTableSwapper().swapToObject(YamlEngine.unmarshal(table, YamlShardingSphereTable.class)));
             }
         });
diff --git a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/ViewMetaDataPersistService.java b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/ViewMetaDataPersistService.java
index d89c8386a71..85f86f735ae 100644
--- a/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/ViewMetaDataPersistService.java
+++ b/mode/core/src/main/java/org/apache/shardingsphere/mode/metadata/persist/service/schema/ViewMetaDataPersistService.java
@@ -17,8 +17,8 @@
 
 package org.apache.shardingsphere.mode.metadata.persist.service.schema;
 
+import com.google.common.base.Strings;
 import lombok.RequiredArgsConstructor;
-import org.apache.commons.lang.StringUtils;
 import org.apache.shardingsphere.infra.metadata.database.schema.decorator.model.ShardingSphereView;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.yaml.schema.pojo.YamlShardingSphereView;
@@ -60,7 +60,7 @@ public final class ViewMetaDataPersistService implements SchemaMetaDataPersistSe
         Map<String, ShardingSphereView> result = new LinkedHashMap<>(viewNames.size(), 1);
         viewNames.forEach(each -> {
             String view = repository.getDirectly(DatabaseMetaDataNode.getViewMetaDataPath(databaseName, schemaName, each));
-            if (!StringUtils.isEmpty(view)) {
+            if (!Strings.isNullOrEmpty(view)) {
                 result.put(each.toLowerCase(), new YamlViewSwapper().swapToObject(YamlEngine.unmarshal(view, YamlShardingSphereView.class)));
             }
         });
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 4fb26347ad8..800ca01b543 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
@@ -34,7 +34,6 @@ import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryFactory;
-import org.apache.shardingsphere.schedule.core.ScheduleContextFactory;
 
 import java.sql.SQLException;
 
@@ -69,8 +68,7 @@ public final class ClusterContextManagerBuilder implements ContextManagerBuilder
     
     private InstanceContext buildInstanceContext(final RegistryCenter registryCenter, final ContextManagerBuilderParameter parameter) {
         return new InstanceContext(new ComputeNodeInstance(parameter.getInstanceMetaData()), new ClusterWorkerIdGenerator(registryCenter, parameter.getInstanceMetaData()),
-                parameter.getModeConfiguration(), new ShardingSphereLockContext(registryCenter.getLockPersistService()),
-                registryCenter.getEventBusContext(), ScheduleContextFactory.newInstance(parameter.getModeConfiguration()));
+                parameter.getModeConfiguration(), new ShardingSphereLockContext(registryCenter.getLockPersistService()), registryCenter.getEventBusContext());
     }
     
     private void persistMetaData(final MetaDataContexts metaDataContexts) {
diff --git a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/ClusterPersistRepositoryFixture.java b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/ClusterPersistRepositoryFixture.java
index 2f596738819..a379d9ab31e 100644
--- a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/ClusterPersistRepositoryFixture.java
+++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/fixture/ClusterPersistRepositoryFixture.java
@@ -17,9 +17,9 @@
 
 package org.apache.shardingsphere.mode.manager.cluster.coordinator.fixture;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;
 
diff --git a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/process/ProcessListClusterPersistRepositoryFixture.java b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/process/ProcessListClusterPersistRepositoryFixture.java
index d204ac93ed6..1c47bb6509d 100644
--- a/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/process/ProcessListClusterPersistRepositoryFixture.java
+++ b/mode/type/cluster/core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/process/ProcessListClusterPersistRepositoryFixture.java
@@ -17,9 +17,9 @@
 
 package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.process;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;
 
diff --git a/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/ClusterPersistRepository.java b/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/ClusterPersistRepository.java
index 439cfe58fe8..0c042b7c5e3 100644
--- a/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/ClusterPersistRepository.java
+++ b/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/ClusterPersistRepository.java
@@ -17,7 +17,6 @@
 
 package org.apache.shardingsphere.mode.repository.cluster;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.persist.PersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;
diff --git a/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/LeaderExecutionCallback.java b/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/LeaderExecutionCallback.java
new file mode 100644
index 00000000000..4270cb156ae
--- /dev/null
+++ b/mode/type/cluster/repository/api/src/main/java/org/apache/shardingsphere/mode/repository/cluster/LeaderExecutionCallback.java
@@ -0,0 +1,29 @@
+/*
+ * 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.repository.cluster;
+
+/**
+ * Leader server execution callback.
+ */
+public interface LeaderExecutionCallback {
+    
+    /**
+     * Execute after leader elected.
+     */
+    void execute();
+}
diff --git a/mode/type/cluster/repository/provider/consul/pom.xml b/mode/type/cluster/repository/provider/consul/pom.xml
index 7f8a81c3bab..3096fffb577 100644
--- a/mode/type/cluster/repository/provider/consul/pom.xml
+++ b/mode/type/cluster/repository/provider/consul/pom.xml
@@ -49,5 +49,9 @@
             <artifactId>httpclient</artifactId>
             <version>${httpclient.version}</version>
         </dependency>
+        <dependency>
+            <groupId>com.google.code.gson</groupId>
+            <artifactId>gson</artifactId>
+        </dependency>
     </dependencies>
 </project>
diff --git a/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/ConsulRepository.java b/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/ConsulRepository.java
index 99e4e94b3bb..ac2200f5819 100644
--- a/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/ConsulRepository.java
+++ b/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/ConsulRepository.java
@@ -24,9 +24,9 @@ import com.ecwid.consul.v1.kv.model.GetValue;
 import com.ecwid.consul.v1.kv.model.PutParams;
 import com.ecwid.consul.v1.session.model.NewSession;
 import com.ecwid.consul.v1.session.model.Session;
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.consul.lock.ConsulInternalLockProvider;
 import org.apache.shardingsphere.mode.repository.cluster.consul.props.ConsulProperties;
 import org.apache.shardingsphere.mode.repository.cluster.consul.props.ConsulPropertyKey;
diff --git a/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/lock/ConsulInternalLockProvider.java b/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/lock/ConsulInternalLockProvider.java
index 3f5a8551301..8610f659215 100644
--- a/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/lock/ConsulInternalLockProvider.java
+++ b/mode/type/cluster/repository/provider/consul/src/main/java/org/apache/shardingsphere/mode/repository/cluster/consul/lock/ConsulInternalLockProvider.java
@@ -28,10 +28,10 @@ import com.ecwid.consul.v1.kv.model.GetValue;
 import com.ecwid.consul.v1.kv.model.PutParams;
 import com.ecwid.consul.v1.session.model.NewSession;
 import com.ecwid.consul.v1.session.model.Session;
-import com.google.gson.reflect.TypeToken;
+import com.google.common.base.Strings;
+import com.google.common.reflect.TypeToken;
 import lombok.RequiredArgsConstructor;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.commons.lang.StringUtils;
 import org.apache.shardingsphere.mode.repository.cluster.consul.ShardingSphereConsulClient;
 import org.apache.shardingsphere.mode.repository.cluster.consul.ShardingSphereQueryParams;
 import org.apache.shardingsphere.mode.repository.cluster.consul.props.ConsulProperties;
@@ -135,7 +135,7 @@ public class ConsulInternalLockProvider implements InternalLockProvider {
         
         @Override
         public boolean tryLock(final long timeoutMillis) {
-            if (StringUtils.isNotEmpty(lockSessionMap.get())) {
+            if (!Strings.isNullOrEmpty(lockSessionMap.get())) {
                 return true;
             }
             try {
diff --git a/mode/type/cluster/repository/provider/etcd/src/main/java/org/apache/shardingsphere/mode/repository/cluster/etcd/EtcdRepository.java b/mode/type/cluster/repository/provider/etcd/src/main/java/org/apache/shardingsphere/mode/repository/cluster/etcd/EtcdRepository.java
index 7fcfcf7572b..d4a3d0427cc 100644
--- a/mode/type/cluster/repository/provider/etcd/src/main/java/org/apache/shardingsphere/mode/repository/cluster/etcd/EtcdRepository.java
+++ b/mode/type/cluster/repository/provider/etcd/src/main/java/org/apache/shardingsphere/mode/repository/cluster/etcd/EtcdRepository.java
@@ -31,9 +31,9 @@ import io.etcd.jetcd.support.Observers;
 import io.etcd.jetcd.watch.WatchEvent;
 import lombok.SneakyThrows;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.etcd.lock.EtcdInternalLockProvider;
 import org.apache.shardingsphere.mode.repository.cluster.etcd.props.EtcdProperties;
 import org.apache.shardingsphere.mode.repository.cluster.etcd.props.EtcdPropertyKey;
diff --git a/mode/type/cluster/repository/provider/nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java b/mode/type/cluster/repository/provider/nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java
index 2dd2e46432a..6b463020e33 100644
--- a/mode/type/cluster/repository/provider/nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java
+++ b/mode/type/cluster/repository/provider/nacos/src/main/java/org/apache/shardingsphere/mode/repository/cluster/nacos/NacosRepository.java
@@ -27,10 +27,10 @@ import com.alibaba.nacos.common.utils.StringUtils;
 import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import lombok.SneakyThrows;
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.infra.instance.utils.IpUtils;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.exception.ClusterPersistRepositoryException;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.nacos.entity.KeyValue;
diff --git a/mode/type/cluster/repository/provider/zookeeper-curator/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/CuratorZookeeperRepository.java b/mode/type/cluster/repository/provider/zookeeper-curator/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/CuratorZookeeperRepository.java
index 671cc4f3ed0..98125f0855b 100644
--- a/mode/type/cluster/repository/provider/zookeeper-curator/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/CuratorZookeeperRepository.java
+++ b/mode/type/cluster/repository/provider/zookeeper-curator/src/main/java/org/apache/shardingsphere/mode/repository/cluster/zookeeper/CuratorZookeeperRepository.java
@@ -31,11 +31,11 @@ import org.apache.curator.framework.recipes.cache.CuratorCacheListener;
 import org.apache.curator.framework.recipes.cache.TreeCacheEvent;
 import org.apache.curator.retry.ExponentialBackoffRetry;
 import org.apache.curator.utils.CloseableUtils;
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.infra.instance.InstanceContext;
 import org.apache.shardingsphere.infra.instance.InstanceContextAware;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.exception.ClusterPersistRepositoryException;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEvent.Type;
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 73b6e893ac9..638ff390275 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
@@ -31,7 +31,6 @@ import org.apache.shardingsphere.mode.metadata.MetaDataContextsFactory;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepository;
 import org.apache.shardingsphere.mode.repository.standalone.StandalonePersistRepositoryFactory;
-import org.apache.shardingsphere.schedule.core.ScheduleContextFactory;
 
 import java.sql.SQLException;
 
@@ -59,8 +58,7 @@ public final class StandaloneContextManagerBuilder implements ContextManagerBuil
     
     private InstanceContext buildInstanceContext(final ContextManagerBuilderParameter parameter) {
         return new InstanceContext(new ComputeNodeInstance(parameter.getInstanceMetaData()),
-                new StandaloneWorkerIdGenerator(), parameter.getModeConfiguration(), new ShardingSphereLockContext(null),
-                new EventBusContext(), ScheduleContextFactory.newInstance(parameter.getModeConfiguration()));
+                new StandaloneWorkerIdGenerator(), parameter.getModeConfiguration(), new ShardingSphereLockContext(null), new EventBusContext());
     }
     
     @Override
diff --git a/mode/type/standalone/repository/provider/jdbc/core/src/test/java/org/apache/shardingsphere/mode/repository/standalone/jdbc/JDBCRepositoryTest.java b/mode/type/standalone/repository/provider/jdbc/core/src/test/java/org/apache/shardingsphere/mode/repository/standalone/jdbc/JDBCRepositoryTest.java
index 9c6c91805cf..f20d0e53168 100644
--- a/mode/type/standalone/repository/provider/jdbc/core/src/test/java/org/apache/shardingsphere/mode/repository/standalone/jdbc/JDBCRepositoryTest.java
+++ b/mode/type/standalone/repository/provider/jdbc/core/src/test/java/org/apache/shardingsphere/mode/repository/standalone/jdbc/JDBCRepositoryTest.java
@@ -31,7 +31,6 @@ import com.zaxxer.hikari.HikariDataSource;
 import java.sql.SQLException;
 import java.util.List;
 import java.util.Properties;
-import org.apache.commons.lang3.StringUtils;
 import org.apache.shardingsphere.mode.repository.standalone.jdbc.fixture.JDBCRepositoryProviderFixture;
 import org.h2.jdbc.JdbcCallableStatement;
 import org.h2.jdbc.JdbcConnection;
@@ -159,7 +158,7 @@ public final class JDBCRepositoryTest {
         when(mockPreparedStatement.executeQuery()).thenReturn(mockResultSet);
         when(mockResultSet.next()).thenReturn(false);
         repository.persist(key, value);
-        int depthOfDirectory = StringUtils.countMatches(key, "/");
+        int depthOfDirectory = (int) key.chars().filter(ch -> ch == '/').count();
         int beginIndex = 0;
         String parentDirectory = "/";
         for (int i = 0; i < depthOfDirectory; i++) {
diff --git a/proxy/bootstrap/src/test/java/org/apache/shardingsphere/proxy/fixture/ClusterPersistRepositoryFixture.java b/proxy/bootstrap/src/test/java/org/apache/shardingsphere/proxy/fixture/ClusterPersistRepositoryFixture.java
index 9df2694a136..9c95e5f2370 100644
--- a/proxy/bootstrap/src/test/java/org/apache/shardingsphere/proxy/fixture/ClusterPersistRepositoryFixture.java
+++ b/proxy/bootstrap/src/test/java/org/apache/shardingsphere/proxy/fixture/ClusterPersistRepositoryFixture.java
@@ -17,9 +17,9 @@
 
 package org.apache.shardingsphere.proxy.fixture;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;
 
diff --git a/test/integration-driver-test/src/test/java/org/apache/shardingsphere/driver/fixture/TestClusterPersistRepository.java b/test/integration-driver-test/src/test/java/org/apache/shardingsphere/driver/fixture/TestClusterPersistRepository.java
index cf5d31e9c66..cd3d79abac0 100644
--- a/test/integration-driver-test/src/test/java/org/apache/shardingsphere/driver/fixture/TestClusterPersistRepository.java
+++ b/test/integration-driver-test/src/test/java/org/apache/shardingsphere/driver/fixture/TestClusterPersistRepository.java
@@ -17,10 +17,10 @@
 
 package org.apache.shardingsphere.driver.fixture;
 
-import org.apache.shardingsphere.elasticjob.lite.internal.storage.LeaderExecutionCallback;
 import org.apache.shardingsphere.infra.database.DefaultDatabase;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
+import org.apache.shardingsphere.mode.repository.cluster.LeaderExecutionCallback;
 import org.apache.shardingsphere.mode.repository.cluster.listener.DataChangedEventListener;
 import org.apache.shardingsphere.mode.repository.cluster.transaction.TransactionOperation;