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

[shardingsphere] branch master updated: Refactor mode-core and discovery-module and readwrite-module dependency (#17513)

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 46c1d053902 Refactor mode-core and discovery-module and readwrite-module dependency (#17513)
46c1d053902 is described below

commit 46c1d05390205ff203fe1d9a9fcd6fe5ad44f2bd
Author: zhaojinchao <zh...@apache.org>
AuthorDate: Tue May 10 15:17:03 2022 +0800

    Refactor mode-core and discovery-module and readwrite-module dependency (#17513)
    
    * Refactor mode-core and discovery-module and readwrite-module dependencie
    
    * Fix checkstyle
---
 .../shardingsphere-db-discovery-core/pom.xml                   |  2 +-
 .../dbdiscovery/algorithm/DatabaseDiscoveryEngine.java         | 10 +++++-----
 .../shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java |  6 +++---
 .../shardingsphere-readwrite-splitting-core/pom.xml            |  5 +++++
 .../readwritesplitting/rule/ReadwriteSplittingRule.java        |  2 +-
 .../readwritesplitting/rule/ReadwriteSplittingRuleTest.java    |  2 +-
 .../org/apache/shardingsphere/schedule/core/api}/CronJob.java  |  2 +-
 .../shardingsphere/schedule/core/api/ModeScheduleContext.java  |  1 -
 .../mode/metadata}/storage/StorageNodeDataSource.java          |  2 +-
 .../shardingsphere/mode/metadata}/storage/StorageNodeRole.java |  2 +-
 .../mode/metadata}/storage/StorageNodeStatus.java              |  2 +-
 .../mode/metadata/storage/event}/DataSourceDisabledEvent.java  |  4 ++--
 .../metadata/storage/event}/DataSourceNameDisabledEvent.java   |  2 +-
 .../metadata/storage/event}/PrimaryDataSourceChangedEvent.java |  2 +-
 .../cluster/coordinator/ClusterContextManagerCoordinator.java  |  8 ++++----
 .../status/storage/service/StorageNodeStatusService.java       |  2 +-
 .../status/storage/subscriber/StorageNodeStatusSubscriber.java | 10 +++++-----
 .../status/storage/watcher/StorageNodeStateChangedWatcher.java |  6 +++---
 .../coordinator/ClusterContextManagerCoordinatorTest.java      |  2 +-
 .../storage/subscriber/StorageNodeStatusSubscriberTest.java    | 10 +++++-----
 .../queryable/ShowReadwriteSplittingReadResourcesHandler.java  |  4 ++--
 .../common/updatable/SetReadwriteSplittingStatusHandler.java   |  8 ++++----
 22 files changed, 49 insertions(+), 45 deletions(-)

diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/pom.xml b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/pom.xml
index 3d62aa8c1a3..163c93b182d 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/pom.xml
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/pom.xml
@@ -60,7 +60,7 @@
         </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
-            <artifactId>shardingsphere-schedule-core</artifactId>
+            <artifactId>shardingsphere-mode-core</artifactId>
             <version>${project.version}</version>
         </dependency>
         <dependency>
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/algorithm/DatabaseDiscoveryEngine.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/algorithm/DatabaseDiscoveryEngine.java
index 2318d1441b8..5f69aa19145 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/algorithm/DatabaseDiscoveryEngine.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/algorithm/DatabaseDiscoveryEngine.java
@@ -26,11 +26,11 @@ import org.apache.shardingsphere.infra.executor.kernel.ExecutorEngine;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroup;
 import org.apache.shardingsphere.infra.executor.kernel.model.ExecutionGroupContext;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceDisabledEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceChangedEvent;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeRole;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
 
 import javax.sql.DataSource;
 import java.sql.SQLException;
diff --git a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
index cb66671583d..4c426301305 100644
--- a/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
+++ b/shardingsphere-features/shardingsphere-db-discovery/shardingsphere-db-discovery-core/src/main/java/org/apache/shardingsphere/dbdiscovery/rule/DatabaseDiscoveryRule.java
@@ -32,13 +32,13 @@ import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmC
 import org.apache.shardingsphere.infra.distsql.constant.ExportableConstants;
 import org.apache.shardingsphere.infra.exception.ShardingSphereException;
 import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceChangedEvent;
 import org.apache.shardingsphere.infra.rule.identifier.scope.SchemaRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.ExportableRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
-import org.apache.shardingsphere.infra.schedule.CronJob;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceNameDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
+import org.apache.shardingsphere.schedule.core.api.CronJob;
 import org.apache.shardingsphere.schedule.core.api.ModeScheduleContext;
 import org.apache.shardingsphere.schedule.core.api.ModeScheduleContextFactory;
 
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/pom.xml b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/pom.xml
index bcb4361576a..2a28a588cc9 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/pom.xml
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/pom.xml
@@ -49,6 +49,11 @@
             <artifactId>shardingsphere-infra-route</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>org.apache.shardingsphere</groupId>
+            <artifactId>shardingsphere-mode-core</artifactId>
+            <version>${project.version}</version>
+        </dependency>
         <dependency>
             <groupId>org.apache.shardingsphere</groupId>
             <artifactId>shardingsphere-transaction-core</artifactId>
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
index 9b81cf5c2d2..1329a48eb13 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/main/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRule.java
@@ -21,11 +21,11 @@ import com.google.common.base.Preconditions;
 import com.google.common.base.Strings;
 import org.apache.shardingsphere.infra.distsql.constant.ExportableConstants;
 import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.infra.rule.identifier.scope.SchemaRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.DataSourceContainedRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.ExportableRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.readwritesplitting.algorithm.config.AlgorithmProvidedReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.api.rule.ReadwriteSplittingDataSourceRuleConfiguration;
diff --git a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
index aefc2b915c1..9b2b46ae626 100644
--- a/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
+++ b/shardingsphere-features/shardingsphere-readwrite-splitting/shardingsphere-readwrite-splitting-core/src/test/java/org/apache/shardingsphere/readwritesplitting/rule/ReadwriteSplittingRuleTest.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.readwritesplitting.rule;
 import com.google.common.collect.ImmutableMap;
 import org.apache.shardingsphere.infra.config.algorithm.ShardingSphereAlgorithmConfiguration;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.readwritesplitting.api.ReadwriteSplittingRuleConfiguration;
 import org.apache.shardingsphere.readwritesplitting.api.rule.ReadwriteSplittingDataSourceRuleConfiguration;
 import org.junit.Test;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/schedule/CronJob.java b/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/CronJob.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/schedule/CronJob.java
rename to shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/CronJob.java
index 04c0089687f..dbf840ee13c 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/schedule/CronJob.java
+++ b/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/CronJob.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.schedule;
+package org.apache.shardingsphere.schedule.core.api;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/ModeScheduleContext.java b/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/ModeScheduleContext.java
index 3837caa06c6..21ad1c730fe 100644
--- a/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/ModeScheduleContext.java
+++ b/shardingsphere-kernel/shardingsphere-schedule/shardingsphere-schedule-core/src/main/java/org/apache/shardingsphere/schedule/core/api/ModeScheduleContext.java
@@ -29,7 +29,6 @@ import org.apache.shardingsphere.elasticjob.reg.zookeeper.ZookeeperConfiguration
 import org.apache.shardingsphere.elasticjob.reg.zookeeper.ZookeeperRegistryCenter;
 import org.apache.shardingsphere.elasticjob.simple.job.SimpleJob;
 import org.apache.shardingsphere.infra.config.mode.ModeConfiguration;
-import org.apache.shardingsphere.infra.schedule.CronJob;
 
 import java.util.HashMap;
 import java.util.Map;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeDataSource.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeDataSource.java
similarity index 96%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeDataSource.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeDataSource.java
index 9abe1ef645a..028ca48e040 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeDataSource.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeDataSource.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.storage;
+package org.apache.shardingsphere.mode.metadata.storage;
 
 import lombok.Getter;
 import lombok.NoArgsConstructor;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeRole.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeRole.java
similarity index 93%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeRole.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeRole.java
index acac654ceec..dbdf4165787 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeRole.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeRole.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.storage;
+package org.apache.shardingsphere.mode.metadata.storage;
 
 /**
  * Storage node role.
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeStatus.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeStatus.java
similarity index 94%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeStatus.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeStatus.java
index 6bc02890ea8..ade0e293154 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/storage/StorageNodeStatus.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/StorageNodeStatus.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.storage;
+package org.apache.shardingsphere.mode.metadata.storage;
 
 /**
  * Storage node status.
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceDisabledEvent.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceDisabledEvent.java
similarity index 90%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceDisabledEvent.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceDisabledEvent.java
index 050e287b1ee..b8cf5a24589 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceDisabledEvent.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceDisabledEvent.java
@@ -15,12 +15,12 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.rule.event.impl;
+package org.apache.shardingsphere.mode.metadata.storage.event;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
 import org.apache.shardingsphere.infra.rule.event.DataSourceStatusChangedEvent;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
 
 /**
  * Data source disabled event.
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceNameDisabledEvent.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceNameDisabledEvent.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceNameDisabledEvent.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceNameDisabledEvent.java
index eadab629a51..94741bd0ba7 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/DataSourceNameDisabledEvent.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/DataSourceNameDisabledEvent.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.rule.event.impl;
+package org.apache.shardingsphere.mode.metadata.storage.event;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/PrimaryDataSourceChangedEvent.java b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/PrimaryDataSourceChangedEvent.java
similarity index 95%
rename from shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/PrimaryDataSourceChangedEvent.java
rename to shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/PrimaryDataSourceChangedEvent.java
index fd65d2f7617..eb4008d21a0 100644
--- a/shardingsphere-infra/shardingsphere-infra-common/src/main/java/org/apache/shardingsphere/infra/rule/event/impl/PrimaryDataSourceChangedEvent.java
+++ b/shardingsphere-mode/shardingsphere-mode-core/src/main/java/org/apache/shardingsphere/mode/metadata/storage/event/PrimaryDataSourceChangedEvent.java
@@ -15,7 +15,7 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.infra.rule.event.impl;
+package org.apache.shardingsphere.mode.metadata.storage.event;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
index ae39a13fdcc..09b39fcb6b2 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinator.java
@@ -26,12 +26,10 @@ import org.apache.shardingsphere.infra.executor.sql.process.model.yaml.BatchYaml
 import org.apache.shardingsphere.infra.instance.ComputeNodeInstance;
 import org.apache.shardingsphere.infra.instance.definition.InstanceDefinition;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceChangedEvent;
 import org.apache.shardingsphere.infra.rule.identifier.type.InstanceAwareRule;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.config.event.datasource.DataSourceChangedEvent;
@@ -59,6 +57,8 @@ import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.statu
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.event.PrimaryStateChangedEvent;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
 import org.apache.shardingsphere.mode.metadata.persist.node.ComputeNode;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceNameDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
 
 import java.sql.SQLException;
 import java.util.Collection;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/service/StorageNodeStatusService.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/service/StorageNodeStatusService.java
index c53aa6ded8f..d7828f1dfbe 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/service/StorageNodeStatusService.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/service/StorageNodeStatusService.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.stat
 
 import com.google.common.base.Strings;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.node.StorageStatusNode;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriber.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatus [...]
index 0e34765a3a0..e92569c1af8 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriber.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriber.java
@@ -19,14 +19,14 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.stat
 
 import com.google.common.eventbus.Subscribe;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceDisabledEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceChangedEvent;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeRole;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.node.StorageStatusNode;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 
 /**
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/watcher/StorageNodeStateChangedWatcher.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/watcher/StorageNodeStateChan [...]
index c21237dc96a..6f158eea577 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/watcher/StorageNodeStateChangedWatcher.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/main/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/watcher/StorageNodeStateChangedWatcher.java
@@ -19,9 +19,9 @@ package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.stat
 
 import com.google.common.base.Strings;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeRole;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceEvent;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.GovernanceWatcher;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
index fab2b57e2c1..25ef7020365 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/ClusterContextManagerCoordinatorTest.java
@@ -47,7 +47,6 @@ import org.apache.shardingsphere.infra.metadata.schema.ShardingSphereSchema;
 import org.apache.shardingsphere.infra.metadata.schema.model.TableMetaData;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.rule.ShardingSphereRule;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.infra.rule.identifier.type.StatusContainedRule;
 import org.apache.shardingsphere.infra.state.StateType;
 import org.apache.shardingsphere.mode.manager.ContextManager;
@@ -76,6 +75,7 @@ import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.statu
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.util.ReflectionUtil;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceNameDisabledEvent;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepositoryConfiguration;
 import org.apache.shardingsphere.parser.rule.SQLParserRule;
diff --git a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriberTest.java b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeSt [...]
index 8cfd988a36e..a5eabae9fb5 100644
--- a/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriberTest.java
+++ b/shardingsphere-mode/shardingsphere-mode-type/shardingsphere-cluster-mode/shardingsphere-cluster-mode-core/src/test/java/org/apache/shardingsphere/mode/manager/cluster/coordinator/registry/status/storage/subscriber/StorageNodeStatusSubscriberTest.java
@@ -18,13 +18,13 @@
 package org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.subscriber;
 
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceDisabledEvent;
-import org.apache.shardingsphere.infra.rule.event.impl.PrimaryDataSourceChangedEvent;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeRole;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.infra.yaml.engine.YamlEngine;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.node.StorageStatusNode;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceDisabledEvent;
+import org.apache.shardingsphere.mode.metadata.storage.event.PrimaryDataSourceChangedEvent;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.junit.Test;
 import org.junit.runner.RunWith;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowReadwriteSplittingReadResourcesHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowReadwriteSplittingReadResourcesHandler.java
index b817cb175ed..0115119bbe9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowReadwriteSplittingReadResourcesHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/queryable/ShowReadwriteSplittingReadResourcesHandler.java
@@ -23,8 +23,8 @@ import org.apache.shardingsphere.infra.exception.DatabaseNotExistedException;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
 import org.apache.shardingsphere.infra.rule.identifier.type.ExportableRule;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.service.StorageNodeStatusService;
 import org.apache.shardingsphere.mode.metadata.MetaDataContexts;
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
index f603278da99..c1b0a307c3d 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/ral/common/updatable/SetReadwriteSplittingStatusHandler.java
@@ -25,13 +25,13 @@ import org.apache.shardingsphere.infra.distsql.exception.resource.RequiredResour
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.exception.DatabaseNotExistedException;
 import org.apache.shardingsphere.infra.metadata.schema.QualifiedDatabase;
-import org.apache.shardingsphere.infra.rule.event.impl.DataSourceDisabledEvent;
-import org.apache.shardingsphere.infra.storage.StorageNodeDataSource;
-import org.apache.shardingsphere.infra.storage.StorageNodeRole;
-import org.apache.shardingsphere.infra.storage.StorageNodeStatus;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeDataSource;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeRole;
+import org.apache.shardingsphere.mode.metadata.storage.StorageNodeStatus;
 import org.apache.shardingsphere.mode.manager.ContextManager;
 import org.apache.shardingsphere.mode.manager.cluster.coordinator.registry.status.storage.service.StorageNodeStatusService;
 import org.apache.shardingsphere.mode.metadata.persist.MetaDataPersistService;
+import org.apache.shardingsphere.mode.metadata.storage.event.DataSourceDisabledEvent;
 import org.apache.shardingsphere.mode.repository.cluster.ClusterPersistRepository;
 import org.apache.shardingsphere.proxy.backend.context.ProxyContext;
 import org.apache.shardingsphere.proxy.backend.exception.NoDatabaseSelectedException;