You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by qi...@apache.org on 2021/02/02 10:13:40 UTC

[shardingsphere] branch master updated: Refactor event for governance and RDL (#9271)

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

qiulu 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 4fba235  Refactor event for governance and RDL (#9271)
4fba235 is described below

commit 4fba235b44251a01bf0f24723ab47087485004a6
Author: Haoran Meng <me...@gmail.com>
AuthorDate: Tue Feb 2 18:13:08 2021 +0800

    Refactor event for governance and RDL (#9271)
---
 .../governance/core/config/ConfigCenter.java       | 12 +++++------
 .../model/rule/RuleConfigurationsAlteredEvent.java | 17 ++++-----------
 .../StartScalingEvent.java}                        | 24 ++++++++++++++--------
 .../governance/core/config/ConfigCenterTest.java   |  4 ++--
 .../rdl/AlterReplicaQueryRuleBackendHandler.java   |  4 ++--
 .../rdl/AlterShardingRuleBackendHandler.java       |  4 ++--
 .../impl/CreateReplicaQueryRuleBackendHandler.java |  4 ++--
 .../rdl/impl/CreateShardingRuleBackendHandler.java |  4 ++--
 .../impl/DropReplicaQueryRuleBackendHandler.java   |  4 ++--
 .../rdl/impl/DropShardingRuleBackendHandler.java   |  4 ++--
 .../scaling/core/api/ScalingWorker.java            |  8 ++++----
 11 files changed, 44 insertions(+), 45 deletions(-)

diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
index a5c9c8a..2ed5d8f 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/config/ConfigCenter.java
@@ -29,8 +29,8 @@ import org.apache.shardingsphere.governance.core.event.model.metadata.MetaDataCr
 import org.apache.shardingsphere.governance.core.event.model.metadata.MetaDataDroppedEvent;
 import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationCachedEvent;
 import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
 import org.apache.shardingsphere.governance.core.event.model.rule.SwitchRuleConfigurationEvent;
+import org.apache.shardingsphere.governance.core.event.model.scaling.StartScalingEvent;
 import org.apache.shardingsphere.governance.core.event.model.schema.SchemaAlteredEvent;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlConfigurationConverter;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlDataSourceConfiguration;
@@ -130,10 +130,10 @@ public final class ConfigCenter {
     /**
      * Persist rule configurations.
      * 
-     * @param event Rule event.
+     * @param event rule configurations altered event.
      */
     @Subscribe
-    public synchronized void renew(final RuleConfigurationsPersistEvent event) {
+    public synchronized void renew(final RuleConfigurationsAlteredEvent event) {
         //TODO
         persistRuleConfigurations(event.getSchemaName(), event.getRuleConfigurations());
     }
@@ -196,11 +196,11 @@ public final class ConfigCenter {
      */
     @Subscribe
     public synchronized void renew(final RuleConfigurationCachedEvent event) {
-        RuleConfigurationsAlteredEvent ruleConfigurationsAlteredEvent = new RuleConfigurationsAlteredEvent(event.getSchemaName(),
+        StartScalingEvent startScalingEvent = new StartScalingEvent(event.getSchemaName(),
                 repository.get(node.getDataSourcePath(event.getSchemaName())),
                 repository.get(node.getRulePath(event.getSchemaName())),
                 configCacheManager.loadCache(node.getRulePath(event.getSchemaName()), event.getCacheId()), event.getCacheId());
-        ShardingSphereEventBus.getInstance().post(ruleConfigurationsAlteredEvent);
+        ShardingSphereEventBus.getInstance().post(startScalingEvent);
     }
     
     private Collection<RuleConfiguration> loadCachedRuleConfigurations(final String schemaName, final String ruleConfigurationCacheId) {
@@ -244,7 +244,7 @@ public final class ConfigCenter {
     
     private void cacheRuleConfigurations(final String schemaName, final Collection<RuleConfiguration> ruleConfigurations) {
         String cacheId = configCacheManager.cache(node.getRulePath(schemaName), YamlEngine.marshal(createYamlRootRuleConfigurations(schemaName, ruleConfigurations)));
-        RuleConfigurationsAlteredEvent event = new RuleConfigurationsAlteredEvent(schemaName,
+        StartScalingEvent event = new StartScalingEvent(schemaName,
                 repository.get(node.getDataSourcePath(schemaName)),
                 repository.get(node.getRulePath(schemaName)),
                 configCacheManager.loadCache(node.getRulePath(schemaName), cacheId), cacheId);
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsAlteredEvent.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsAlteredEvent.java
index 95ac061..055911f 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsAlteredEvent.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsAlteredEvent.java
@@ -19,6 +19,9 @@ package org.apache.shardingsphere.governance.core.event.model.rule;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.RuleConfiguration;
+
+import java.util.Collection;
 
 /**
  * Rule configurations altered event.
@@ -29,17 +32,5 @@ public final class RuleConfigurationsAlteredEvent {
     
     private final String schemaName;
     
-    private final String sourceDataSource;
-    
-    private final String sourceRule;
-    
-    private final String targetDataSource;
-    
-    private final String targetRule;
-    
-    private final String ruleCacheId;
-    
-    public RuleConfigurationsAlteredEvent(final String schemaName, final String sourceDataSource, final String sourceRule, final String targetRule, final String ruleCacheId) {
-        this(schemaName, sourceDataSource, sourceRule, sourceDataSource, targetRule, ruleCacheId);
-    }
+    private final Collection<RuleConfiguration> ruleConfigurations;
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsPersistEvent.java b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/scaling/StartScalingEvent.java
similarity index 60%
rename from shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsPersistEvent.java
rename to shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/scaling/StartScalingEvent.java
index f100051..664b3f2 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/rule/RuleConfigurationsPersistEvent.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/main/java/org/apache/shardingsphere/governance/core/event/model/scaling/StartScalingEvent.java
@@ -15,23 +15,31 @@
  * limitations under the License.
  */
 
-package org.apache.shardingsphere.governance.core.event.model.rule;
+package org.apache.shardingsphere.governance.core.event.model.scaling;
 
 import lombok.Getter;
 import lombok.RequiredArgsConstructor;
-import org.apache.shardingsphere.governance.core.event.model.GovernanceEvent;
-import org.apache.shardingsphere.infra.config.RuleConfiguration;
-
-import java.util.Collection;
 
 /**
- * Rule configurations persist event.
+ * Start scaling event.
  */
 @RequiredArgsConstructor
 @Getter
-public final class RuleConfigurationsPersistEvent implements GovernanceEvent {
+public final class StartScalingEvent {
     
     private final String schemaName;
     
-    private final Collection<RuleConfiguration> ruleConfigurations;
+    private final String sourceDataSource;
+    
+    private final String sourceRule;
+    
+    private final String targetDataSource;
+    
+    private final String targetRule;
+    
+    private final String ruleCacheId;
+    
+    public StartScalingEvent(final String schemaName, final String sourceDataSource, final String sourceRule, final String targetRule, final String ruleCacheId) {
+        this(schemaName, sourceDataSource, sourceRule, sourceDataSource, targetRule, ruleCacheId);
+    }
 }
diff --git a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/config/ConfigCenterTest.java b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/config/ConfigCenterTest.java
index 20039c2..ac3017d 100644
--- a/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/config/ConfigCenterTest.java
+++ b/shardingsphere-governance/shardingsphere-governance-core/src/test/java/org/apache/shardingsphere/governance/core/config/ConfigCenterTest.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.governance.core.event.model.datasource.DataSour
 import org.apache.shardingsphere.governance.core.event.model.datasource.DataSourcePersistEvent;
 import org.apache.shardingsphere.governance.core.event.model.metadata.MetaDataCreatedEvent;
 import org.apache.shardingsphere.governance.core.event.model.metadata.MetaDataDroppedEvent;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.governance.core.event.model.rule.SwitchRuleConfigurationEvent;
 import org.apache.shardingsphere.governance.core.event.model.schema.SchemaAlteredEvent;
 import org.apache.shardingsphere.governance.core.yaml.config.YamlRuleConfigurationWrap;
@@ -509,7 +509,7 @@ public final class ConfigCenterTest {
     
     @Test
     public void assertRenewRuleEvent() {
-        RuleConfigurationsPersistEvent event = new RuleConfigurationsPersistEvent("sharding_db", createRuleConfigurations());
+        RuleConfigurationsAlteredEvent event = new RuleConfigurationsAlteredEvent("sharding_db", createRuleConfigurations());
         ConfigCenter configCenter = new ConfigCenter(configurationRepository);
         configCenter.renew(event);
         verify(configurationRepository).persist(startsWith("/metadata/sharding_db/rule"), anyString());
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterReplicaQueryRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterReplicaQueryRuleBackendHandler.java
index d1c6049..c7fc9bb 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterReplicaQueryRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterReplicaQueryRuleBackendHandler.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.rdl;
 
 import org.apache.shardingsphere.distsql.parser.segment.rdl.ReplicaQueryRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterReplicaQueryRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.config.algorithm.YamlShardingSphereAlgorithmConfiguration;
@@ -151,6 +151,6 @@ public final class AlterReplicaQueryRuleBackendHandler extends SchemaRequiredBac
     }
 
     private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, rules));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterShardingRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterShardingRuleBackendHandler.java
index dd01be2..f5b8708 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterShardingRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/AlterShardingRuleBackendHandler.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.rdl;
 import com.google.common.base.Joiner;
 import org.apache.shardingsphere.distsql.parser.segment.TableRuleSegment;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.AlterShardingRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -175,6 +175,6 @@ public final class AlterShardingRuleBackendHandler extends SchemaRequiredBackend
     }
     
     private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, rules));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateReplicaQueryRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateReplicaQueryRuleBackendHandler.java
index 617a20c..2b0eff9 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateReplicaQueryRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateReplicaQueryRuleBackendHandler.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
 
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateReplicaQueryRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -60,6 +60,6 @@ public final class CreateReplicaQueryRuleBackendHandler extends SchemaRequiredBa
     }
     
     private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, rules));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingRuleBackendHandler.java
index 756d9b3..f80c959 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/CreateShardingRuleBackendHandler.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
 
 import org.apache.shardingsphere.distsql.parser.statement.rdl.create.impl.CreateShardingRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -50,7 +50,7 @@ public final class CreateShardingRuleBackendHandler extends SchemaRequiredBacken
     }
     
     private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, rules));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
         // TODO Need to get the executed feedback from registry center for returning.
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropReplicaQueryRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropReplicaQueryRuleBackendHandler.java
index c818e91..7865832 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropReplicaQueryRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropReplicaQueryRuleBackendHandler.java
@@ -18,7 +18,7 @@
 package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
 
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropReplicaQueryRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.config.RuleConfiguration;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.yaml.swapper.YamlRuleConfigurationSwapperEngine;
@@ -85,6 +85,6 @@ public final class DropReplicaQueryRuleBackendHandler extends SchemaRequiredBack
     }
     
     private void post(final String schemaName, final Collection<RuleConfiguration> rules) {
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, rules));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, rules));
     }
 }
diff --git a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java
index 19a03ad..369c36a 100644
--- a/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java
+++ b/shardingsphere-proxy/shardingsphere-proxy-backend/src/main/java/org/apache/shardingsphere/proxy/backend/text/distsql/rdl/impl/DropShardingRuleBackendHandler.java
@@ -19,7 +19,7 @@ package org.apache.shardingsphere.proxy.backend.text.distsql.rdl.impl;
 
 import com.google.common.base.Preconditions;
 import org.apache.shardingsphere.distsql.parser.statement.rdl.drop.impl.DropShardingRuleStatement;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsPersistEvent;
+import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.infra.metadata.ShardingSphereMetaData;
 import org.apache.shardingsphere.proxy.backend.communication.jdbc.connection.BackendConnection;
@@ -112,7 +112,7 @@ public final class DropShardingRuleBackendHandler extends SchemaRequiredBackendH
     
     private void post(final String schemaName) {
         // TODO should use RuleConfigurationsChangeEvent
-        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsPersistEvent(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()));
+        ShardingSphereEventBus.getInstance().post(new RuleConfigurationsAlteredEvent(schemaName, ProxyContext.getInstance().getMetaData(schemaName).getRuleMetaData().getConfigurations()));
         // TODO Need to get the executed feedback from registry center for returning.
     }
 }
diff --git a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/api/ScalingWorker.java b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/api/ScalingWorker.java
index b94ec00..b399215 100644
--- a/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/api/ScalingWorker.java
+++ b/shardingsphere-scaling/shardingsphere-scaling-core/src/main/java/org/apache/shardingsphere/scaling/core/api/ScalingWorker.java
@@ -19,8 +19,8 @@ package org.apache.shardingsphere.scaling.core.api;
 
 import com.google.common.eventbus.Subscribe;
 import lombok.extern.slf4j.Slf4j;
-import org.apache.shardingsphere.governance.core.event.model.rule.RuleConfigurationsAlteredEvent;
 import org.apache.shardingsphere.governance.core.event.model.rule.SwitchRuleConfigurationEvent;
+import org.apache.shardingsphere.governance.core.event.model.scaling.StartScalingEvent;
 import org.apache.shardingsphere.infra.eventbus.ShardingSphereEventBus;
 import org.apache.shardingsphere.scaling.core.config.HandleConfiguration;
 import org.apache.shardingsphere.scaling.core.config.JobConfiguration;
@@ -54,10 +54,10 @@ public final class ScalingWorker {
     /**
      * Start scaling job.
      *
-     * @param event rule configurations altered event.
+     * @param event start scaling event.
      */
     @Subscribe
-    public void start(final RuleConfigurationsAlteredEvent event) {
+    public void start(final StartScalingEvent event) {
         log.info("Start scaling job by {}", event);
         Optional<Long> jobId = scalingAPI.start(createJobConfig(event));
         if (!jobId.isPresent()) {
@@ -66,7 +66,7 @@ public final class ScalingWorker {
         }
     }
     
-    private JobConfiguration createJobConfig(final RuleConfigurationsAlteredEvent event) {
+    private JobConfiguration createJobConfig(final StartScalingEvent event) {
         JobConfiguration result = new JobConfiguration();
         result.setRuleConfig(new RuleConfiguration(
                 new ShardingSphereJDBCDataSourceConfiguration(event.getSourceDataSource(), event.getSourceRule()),