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 2023/06/12 11:18:10 UTC

[shardingsphere] branch master updated: Add feature configuration changed event and subscribe at sharding module (#26304)

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

zhaojinchao 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 5c6d28c1378 Add feature configuration changed event and subscribe at sharding module (#26304)
5c6d28c1378 is described below

commit 5c6d28c1378eb776f1a0e5029d6e81682a7b7c19
Author: ChenJiaHao <Pa...@163.com>
AuthorDate: Mon Jun 12 19:18:00 2023 +0800

    Add feature configuration changed event and subscribe at sharding module (#26304)
    
    * Add feature configuration changed event and subscribe at sharding module
    
    * Resolve conflicts
    
    * Fix UT
---
 .../EncryptRuleConfigurationEventBuilder.java      |   6 +-
 .../event/MaskRuleConfigurationEventBuilder.java   |  12 +-
 .../subscriber/ShadowConfigurationSubscriber.java  |   3 +-
 .../ShardingRuleConfigurationEventBuilder.java     | 310 ++++++++++++++++++
 .../event/algorithm/auditor/AddAuditorEvent.java   |  38 +++
 .../event/algorithm/auditor/AlterAuditorEvent.java |  38 +++
 .../algorithm/auditor/DeleteAuditorEvent.java      |  34 ++
 .../keygenerator/AddKeyGeneratorEvent.java         |  38 +++
 .../keygenerator/AlterKeyGeneratorEvent.java       |  38 +++
 .../keygenerator/DeleteKeyGeneratorEvent.java      |  34 ++
 .../sharding/AddShardingAlgorithmEvent.java        |  38 +++
 .../sharding/AlterShardingAlgorithmEvent.java      |  38 +++
 .../sharding/DeleteShardingAlgorithmEvent.java     |  34 ++
 .../cache/AddShardingCacheConfigurationEvent.java  |  35 ++
 .../AlterShardingCacheConfigurationEvent.java      |  35 ++
 .../DeleteShardingCacheConfigurationEvent.java     |  32 ++
 ...dShardingAuditorStrategyConfigurationEvent.java |  35 ++
 ...rShardingAuditorStrategyConfigurationEvent.java |  35 ++
 ...eShardingAuditorStrategyConfigurationEvent.java |  32 ++
 ...DatabaseShardingStrategyConfigurationEvent.java |  35 ++
 ...DatabaseShardingStrategyConfigurationEvent.java |  35 ++
 ...DatabaseShardingStrategyConfigurationEvent.java |  32 ++
 .../AddKeyGenerateStrategyConfigurationEvent.java  |  35 ++
 ...AlterKeyGenerateStrategyConfigurationEvent.java |  35 ++
 ...eleteKeyGenerateStrategyConfigurationEvent.java |  32 ++
 .../AddDefaultShardingColumnEvent.java             |  34 ++
 .../AlterDefaultShardingColumnEvent.java           |  34 ++
 .../DeleteDefaultShardingColumnEvent.java          |  32 ++
 ...AddTableShardingStrategyConfigurationEvent.java |  35 ++
 ...terTableShardingStrategyConfigurationEvent.java |  35 ++
 ...eteTableShardingStrategyConfigurationEvent.java |  32 ++
 .../AddShardingAutoTableConfigurationEvent.java    |  36 +++
 .../AlterShardingAutoTableConfigurationEvent.java  |  38 +++
 .../DeleteShardingAutoTableConfigurationEvent.java |  34 ++
 ...ddShardingTableReferenceConfigurationEvent.java |  36 +++
 ...erShardingTableReferenceConfigurationEvent.java |  38 +++
 ...teShardingTableReferenceConfigurationEvent.java |  34 ++
 .../AddBroadcastTableConfigurationEvent.java       |  36 +++
 .../AlterBroadcastTableConfigurationEvent.java     |  36 +++
 .../DeleteBroadcastTableConfigurationEvent.java    |  32 ++
 .../AddShardingTableConfigurationEvent.java        |  36 +++
 .../AlterShardingTableConfigurationEvent.java      |  38 +++
 .../DeleteShardingTableConfigurationEvent.java     |  34 ++
 .../metadata/converter/ShardingNodeConverter.java  |  14 +-
 .../subscriber/ShardingAlgorithmSubscriber.java    | 199 ++++++++++++
 .../ShardingCacheConfigurationSubscriber.java      | 101 ++++++
 .../ShardingStrategyConfigurationSubscriber.java   | 355 +++++++++++++++++++++
 .../ShardingTableConfigurationSubscriber.java      | 299 +++++++++++++++++
 .../NewYamlShardingRuleConfigurationSwapper.java   |   2 +-
 ...nfra.rule.RuleConfigurationSubscribeCoordinator |  21 ++
 ...mode.event.config.RuleConfigurationEventBuilder |  18 ++
 .../converter/ShardingNodeConverterTest.java       |  11 +-
 ...ewYamlShardingRuleConfigurationSwapperTest.java |   8 +-
 53 files changed, 2692 insertions(+), 35 deletions(-)

diff --git a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/event/EncryptRuleConfigurationEventBuilder.java b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/event/EncryptRuleConfigurationEventBuilder.java
index b8e395a5091..41cc7cccd5d 100644
--- a/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/event/EncryptRuleConfigurationEventBuilder.java
+++ b/features/encrypt/core/src/main/java/org/apache/shardingsphere/encrypt/event/EncryptRuleConfigurationEventBuilder.java
@@ -26,8 +26,8 @@ import org.apache.shardingsphere.encrypt.event.encryptor.AddEncryptorEvent;
 import org.apache.shardingsphere.encrypt.event.encryptor.AlterEncryptorEvent;
 import org.apache.shardingsphere.encrypt.event.encryptor.DeleteEncryptorEvent;
 import org.apache.shardingsphere.encrypt.metadata.converter.EncryptNodeConverter;
-import org.apache.shardingsphere.encrypt.yaml.config.YamlEncryptRuleConfiguration;
-import org.apache.shardingsphere.encrypt.yaml.swapper.YamlEncryptRuleConfigurationSwapper;
+import org.apache.shardingsphere.encrypt.yaml.config.rule.YamlEncryptTableRuleConfiguration;
+import org.apache.shardingsphere.encrypt.yaml.swapper.rule.YamlEncryptTableRuleConfigurationSwapper;
 import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
 import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
@@ -71,7 +71,7 @@ public final class EncryptRuleConfigurationEventBuilder implements RuleConfigura
     }
     
     private EncryptTableRuleConfiguration swapEncryptTableRuleConfig(final String yamlContext) {
-        return new YamlEncryptRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlEncryptRuleConfiguration.class)).getTables().iterator().next();
+        return new YamlEncryptTableRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlEncryptTableRuleConfiguration.class));
     }
     
     private Optional<GovernanceEvent> createEncryptorEvent(final String databaseName, final String encryptorName, final DataChangedEvent event) {
diff --git a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/event/MaskRuleConfigurationEventBuilder.java b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/event/MaskRuleConfigurationEventBuilder.java
index 0af602f7cd6..50aa50cb27d 100644
--- a/features/mask/core/src/main/java/org/apache/shardingsphere/mask/event/MaskRuleConfigurationEventBuilder.java
+++ b/features/mask/core/src/main/java/org/apache/shardingsphere/mask/event/MaskRuleConfigurationEventBuilder.java
@@ -31,8 +31,8 @@ import org.apache.shardingsphere.mask.event.config.AddMaskConfigurationEvent;
 import org.apache.shardingsphere.mask.event.config.AlterMaskConfigurationEvent;
 import org.apache.shardingsphere.mask.event.config.DeleteMaskConfigurationEvent;
 import org.apache.shardingsphere.mask.metadata.converter.MaskNodeConverter;
-import org.apache.shardingsphere.mask.yaml.config.YamlMaskRuleConfiguration;
-import org.apache.shardingsphere.mask.yaml.swapper.YamlMaskRuleConfigurationSwapper;
+import org.apache.shardingsphere.mask.yaml.config.rule.YamlMaskTableRuleConfiguration;
+import org.apache.shardingsphere.mask.yaml.swapper.rule.YamlMaskTableRuleConfigurationSwapper;
 import org.apache.shardingsphere.mode.event.DataChangedEvent;
 import org.apache.shardingsphere.mode.event.DataChangedEvent.Type;
 import org.apache.shardingsphere.mode.spi.RuleConfigurationEventBuilder;
@@ -60,18 +60,18 @@ public final class MaskRuleConfigurationEventBuilder implements RuleConfiguratio
         return Optional.empty();
     }
     
-    private Optional<GovernanceEvent> createMaskConfigEvent(final String databaseName, final String groupName, final DataChangedEvent event) {
+    private Optional<GovernanceEvent> createMaskConfigEvent(final String databaseName, final String tableName, final DataChangedEvent event) {
         if (Type.ADDED == event.getType()) {
             return Optional.of(new AddMaskConfigurationEvent<>(databaseName, swapMaskTableRuleConfig(event.getValue())));
         }
         if (Type.UPDATED == event.getType()) {
-            return Optional.of(new AlterMaskConfigurationEvent<>(databaseName, groupName, swapMaskTableRuleConfig(event.getValue())));
+            return Optional.of(new AlterMaskConfigurationEvent<>(databaseName, tableName, swapMaskTableRuleConfig(event.getValue())));
         }
-        return Optional.of(new DeleteMaskConfigurationEvent(databaseName, groupName));
+        return Optional.of(new DeleteMaskConfigurationEvent(databaseName, tableName));
     }
     
     private MaskTableRuleConfiguration swapMaskTableRuleConfig(final String yamlContext) {
-        return new YamlMaskRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlMaskRuleConfiguration.class)).getTables().iterator().next();
+        return new YamlMaskTableRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlMaskTableRuleConfiguration.class));
     }
     
     private Optional<GovernanceEvent> createMaskAlgorithmEvent(final String databaseName, final String algorithmName, final DataChangedEvent event) {
diff --git a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/subscriber/ShadowConfigurationSubscriber.java b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/subscriber/ShadowConfigurationSubscriber.java
index be9d6447a59..6cd20f2419c 100644
--- a/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/subscriber/ShadowConfigurationSubscriber.java
+++ b/features/shadow/core/src/main/java/org/apache/shardingsphere/shadow/subscriber/ShadowConfigurationSubscriber.java
@@ -32,7 +32,6 @@ import org.apache.shardingsphere.shadow.event.config.DeleteShadowConfigurationEv
 import org.apache.shardingsphere.shadow.rule.ShadowRule;
 
 import java.util.Collection;
-import java.util.Collections;
 import java.util.LinkedList;
 import java.util.Map;
 import java.util.Optional;
@@ -72,7 +71,7 @@ public final class ShadowConfigurationSubscriber implements RuleConfigurationSub
             config.getDataSources().add(needToAddedConfig);
         } else {
             config = new ShadowRuleConfiguration();
-            config.setDataSources(Collections.singletonList(needToAddedConfig));
+            config.getDataSources().add(needToAddedConfig);
         }
         ruleConfigs.add(config);
         database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/ShardingRuleConfigurationEventBuilder.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/ShardingRuleConfigurationEventBuilder.java
new file mode 100644
index 00000000000..868bcf4dcad
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/ShardingRuleConfigurationEventBuilder.java
@@ -0,0 +1,310 @@
+/*
+ * 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.sharding.event;
+
+import com.google.common.base.Strings;
+import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
+import org.apache.shardingsphere.infra.yaml.config.pojo.algorithm.YamlAlgorithmConfiguration;
+import org.apache.shardingsphere.infra.yaml.config.swapper.algorithm.YamlAlgorithmConfigurationSwapper;
+import org.apache.shardingsphere.mode.event.DataChangedEvent;
+import org.apache.shardingsphere.mode.event.DataChangedEvent.Type;
+import org.apache.shardingsphere.mode.spi.RuleConfigurationEventBuilder;
+import org.apache.shardingsphere.sharding.api.config.cache.ShardingCacheConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableReferenceRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.AddAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.AlterAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.DeleteAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.AddKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.AlterKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.DeleteKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.AddShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.AlterShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.DeleteShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.event.cache.AddShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.cache.AlterShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.cache.DeleteShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.audit.AddShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.audit.AlterShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.audit.DeleteShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.AddDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.AlterDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.DeleteDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.AddKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.AlterKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.DeleteKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.AddDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.AlterDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.DeleteDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.AddTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.AlterTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.DeleteTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.auto.AddShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.auto.AlterShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.auto.DeleteShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.AddBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.AlterBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.DeleteBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.AddShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.AlterShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.DeleteShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.metadata.converter.ShardingNodeConverter;
+import org.apache.shardingsphere.sharding.yaml.config.cache.YamlShardingCacheConfiguration;
+import org.apache.shardingsphere.sharding.yaml.config.rule.YamlShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.yaml.config.rule.YamlTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.yaml.config.strategy.audit.YamlShardingAuditStrategyConfiguration;
+import org.apache.shardingsphere.sharding.yaml.config.strategy.keygen.YamlKeyGenerateStrategyConfiguration;
+import org.apache.shardingsphere.sharding.yaml.config.strategy.sharding.YamlShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.yaml.swapper.cache.YamlShardingCacheConfigurationSwapper;
+import org.apache.shardingsphere.sharding.yaml.swapper.rule.YamlShardingAutoTableRuleConfigurationSwapper;
+import org.apache.shardingsphere.sharding.yaml.swapper.rule.YamlShardingTableReferenceRuleConfigurationConverter;
+import org.apache.shardingsphere.sharding.yaml.swapper.rule.YamlShardingTableRuleConfigurationSwapper;
+import org.apache.shardingsphere.sharding.yaml.swapper.strategy.YamlKeyGenerateStrategyConfigurationSwapper;
+import org.apache.shardingsphere.sharding.yaml.swapper.strategy.YamlShardingAuditStrategyConfigurationSwapper;
+import org.apache.shardingsphere.sharding.yaml.swapper.strategy.YamlShardingStrategyConfigurationSwapper;
+
+import java.util.Collection;
+import java.util.Optional;
+
+/**
+ * Sharding rule configuration event builder.
+ */
+public final class ShardingRuleConfigurationEventBuilder implements RuleConfigurationEventBuilder {
+    
+    @Override
+    public Optional<GovernanceEvent> build(final String databaseName, final DataChangedEvent event) {
+        if (!ShardingNodeConverter.isShardingPath(event.getKey()) || Strings.isNullOrEmpty(event.getValue())) {
+            return Optional.empty();
+        }
+        Optional<String> tableName = ShardingNodeConverter.getTableName(event.getKey());
+        if (tableName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createShardingTableConfigEvent(databaseName, tableName.get(), event);
+        }
+        Optional<String> autoTableName = ShardingNodeConverter.getAutoTableName(event.getKey());
+        if (autoTableName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createShardingAutoTableConfigEvent(databaseName, autoTableName.get(), event);
+        }
+        Optional<String> bindingTableName = ShardingNodeConverter.getBindingTableName(event.getKey());
+        if (bindingTableName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createShardingTableReferenceConfigEvent(databaseName, bindingTableName.get(), event);
+        }
+        if (ShardingNodeConverter.isBroadcastTablePath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createBroadcastTableConfigEvent(databaseName, event);
+        }
+        if (ShardingNodeConverter.isDefaultDatabaseStrategyPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createDefaultDatabaseStrategyConfigEvent(databaseName, event);
+        }
+        if (ShardingNodeConverter.isDefaultTableStrategyPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createDefaultTableStrategyConfigEvent(databaseName, event);
+        }
+        if (ShardingNodeConverter.isDefaultKeyGenerateStrategyPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createDefaultKeyGenerateStrategyConfigEvent(databaseName, event);
+        }
+        if (ShardingNodeConverter.isDefaultAuditStrategyPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createDefaultShardingAuditorStrategyConfigEvent(databaseName, event);
+        }
+        if (ShardingNodeConverter.isDefaultShardingColumnPath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createDefaultShardingColumnEvent(databaseName, event);
+        }
+        Optional<String> algorithmName = ShardingNodeConverter.getShardingAlgorithmName(event.getKey());
+        if (algorithmName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createShardingAlgorithmEvent(databaseName, algorithmName.get(), event);
+        }
+        Optional<String> keyGeneratorName = ShardingNodeConverter.getKeyGeneratorName(event.getKey());
+        if (keyGeneratorName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createKeyGeneratorEvent(databaseName, keyGeneratorName.get(), event);
+        }
+        Optional<String> auditorName = ShardingNodeConverter.getAuditorName(event.getKey());
+        if (auditorName.isPresent() && !Strings.isNullOrEmpty(event.getValue())) {
+            return createAuditorEvent(databaseName, auditorName.get(), event);
+        }
+        if (ShardingNodeConverter.isShardingCachePath(event.getKey()) && !Strings.isNullOrEmpty(event.getValue())) {
+            return createShardingCacheEvent(databaseName, event);
+        }
+        return Optional.empty();
+    }
+    
+    private Optional<GovernanceEvent> createShardingTableConfigEvent(final String databaseName, final String tableName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingTableConfigurationEvent<>(databaseName, swapShardingTableRuleConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingTableConfigurationEvent<>(databaseName, tableName, swapShardingTableRuleConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingTableConfigurationEvent(databaseName, tableName));
+    }
+    
+    private ShardingTableRuleConfiguration swapShardingTableRuleConfig(final String yamlContext) {
+        return new YamlShardingTableRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlTableRuleConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createShardingAutoTableConfigEvent(final String databaseName, final String tableName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingAutoTableConfigurationEvent<>(databaseName, swapShardingAutoTableRuleConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingAutoTableConfigurationEvent<>(databaseName, tableName, swapShardingAutoTableRuleConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingAutoTableConfigurationEvent(databaseName, tableName));
+    }
+    
+    private ShardingAutoTableRuleConfiguration swapShardingAutoTableRuleConfig(final String yamlContext) {
+        return new YamlShardingAutoTableRuleConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlShardingAutoTableRuleConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createShardingTableReferenceConfigEvent(final String databaseName, final String tableName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingAutoTableConfigurationEvent<>(databaseName, swapShardingTableReferenceRuleConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingAutoTableConfigurationEvent<>(databaseName, tableName, swapShardingTableReferenceRuleConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingAutoTableConfigurationEvent(databaseName, tableName));
+    }
+    
+    private ShardingTableReferenceRuleConfiguration swapShardingTableReferenceRuleConfig(final String yamlContext) {
+        return YamlShardingTableReferenceRuleConfigurationConverter.convertToObject(yamlContext);
+    }
+    
+    @SuppressWarnings("unchecked")
+    private Optional<GovernanceEvent> createBroadcastTableConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddBroadcastTableConfigurationEvent(databaseName, YamlEngine.unmarshal(event.getValue(), Collection.class)));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterBroadcastTableConfigurationEvent(databaseName, YamlEngine.unmarshal(event.getValue(), Collection.class)));
+        }
+        return Optional.of(new DeleteBroadcastTableConfigurationEvent(databaseName));
+    }
+    
+    private Optional<GovernanceEvent> createDefaultDatabaseStrategyConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddDatabaseShardingStrategyConfigurationEvent(databaseName, swapShardingStrategyConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterDatabaseShardingStrategyConfigurationEvent(databaseName, swapShardingStrategyConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteDatabaseShardingStrategyConfigurationEvent(databaseName));
+    }
+    
+    private Optional<GovernanceEvent> createDefaultTableStrategyConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddTableShardingStrategyConfigurationEvent(databaseName, swapShardingStrategyConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterTableShardingStrategyConfigurationEvent(databaseName, swapShardingStrategyConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteTableShardingStrategyConfigurationEvent(databaseName));
+    }
+    
+    private ShardingStrategyConfiguration swapShardingStrategyConfig(final String yamlContext) {
+        return new YamlShardingStrategyConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlShardingStrategyConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createDefaultKeyGenerateStrategyConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddKeyGenerateStrategyConfigurationEvent(databaseName, swapKeyGenerateStrategyConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterKeyGenerateStrategyConfigurationEvent(databaseName, swapKeyGenerateStrategyConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteKeyGenerateStrategyConfigurationEvent(databaseName));
+    }
+    
+    private KeyGenerateStrategyConfiguration swapKeyGenerateStrategyConfig(final String yamlContext) {
+        return new YamlKeyGenerateStrategyConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlKeyGenerateStrategyConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createDefaultShardingAuditorStrategyConfigEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingAuditorStrategyConfigurationEvent(databaseName, swapShardingAuditorStrategyConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingAuditorStrategyConfigurationEvent(databaseName, swapShardingAuditorStrategyConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingAuditorStrategyConfigurationEvent(databaseName));
+    }
+    
+    private ShardingAuditStrategyConfiguration swapShardingAuditorStrategyConfig(final String yamlContext) {
+        return new YamlShardingAuditStrategyConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlShardingAuditStrategyConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createDefaultShardingColumnEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddDefaultShardingColumnEvent(databaseName, event.getValue()));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterDefaultShardingColumnEvent(databaseName, event.getValue()));
+        }
+        return Optional.of(new DeleteDefaultShardingColumnEvent(databaseName));
+    }
+    
+    private Optional<GovernanceEvent> createShardingAlgorithmEvent(final String databaseName, final String algorithmName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingAlgorithmEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingAlgorithmEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingAlgorithmEvent(databaseName, algorithmName));
+    }
+    
+    private Optional<GovernanceEvent> createKeyGeneratorEvent(final String databaseName, final String algorithmName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddKeyGeneratorEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterKeyGeneratorEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteKeyGeneratorEvent(databaseName, algorithmName));
+    }
+    
+    private Optional<GovernanceEvent> createAuditorEvent(final String databaseName, final String algorithmName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddAuditorEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterAuditorEvent<>(databaseName, algorithmName, swapToAlgorithmConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteAuditorEvent(databaseName, algorithmName));
+    }
+    
+    private AlgorithmConfiguration swapToAlgorithmConfig(final String yamlContext) {
+        return new YamlAlgorithmConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlAlgorithmConfiguration.class));
+    }
+    
+    private Optional<GovernanceEvent> createShardingCacheEvent(final String databaseName, final DataChangedEvent event) {
+        if (Type.ADDED == event.getType()) {
+            return Optional.of(new AddShardingCacheConfigurationEvent(databaseName, swapToShardingCacheConfig(event.getValue())));
+        }
+        if (Type.UPDATED == event.getType()) {
+            return Optional.of(new AlterShardingCacheConfigurationEvent(databaseName, swapToShardingCacheConfig(event.getValue())));
+        }
+        return Optional.of(new DeleteShardingCacheConfigurationEvent(databaseName));
+    }
+    
+    private ShardingCacheConfiguration swapToShardingCacheConfig(final String yamlContext) {
+        return new YamlShardingCacheConfigurationSwapper().swapToObject(YamlEngine.unmarshal(yamlContext, YamlShardingCacheConfiguration.class));
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AddAuditorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AddAuditorEvent.java
new file mode 100644
index 00000000000..5219e837cb3
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AddAuditorEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.auditor;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add auditor event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddAuditorEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String auditorName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AlterAuditorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AlterAuditorEvent.java
new file mode 100644
index 00000000000..74520da57fa
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/AlterAuditorEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.auditor;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter auditor event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterAuditorEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String auditorName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/DeleteAuditorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/DeleteAuditorEvent.java
new file mode 100644
index 00000000000..0d18f6f42cb
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/auditor/DeleteAuditorEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.algorithm.auditor;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete auditor event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteAuditorEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String auditorName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AddKeyGeneratorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AddKeyGeneratorEvent.java
new file mode 100644
index 00000000000..020a47ce58d
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AddKeyGeneratorEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.keygenerator;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add key generate event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddKeyGeneratorEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String keyGeneratorName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AlterKeyGeneratorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AlterKeyGeneratorEvent.java
new file mode 100644
index 00000000000..0cc014d4ab4
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/AlterKeyGeneratorEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.keygenerator;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter key generate event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterKeyGeneratorEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String keyGeneratorName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/DeleteKeyGeneratorEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/DeleteKeyGeneratorEvent.java
new file mode 100644
index 00000000000..a56bfa0d959
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/keygenerator/DeleteKeyGeneratorEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.algorithm.keygenerator;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete key generate event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteKeyGeneratorEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String keyGeneratorName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AddShardingAlgorithmEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AddShardingAlgorithmEvent.java
new file mode 100644
index 00000000000..36b30d80d90
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AddShardingAlgorithmEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add sharding algorithm event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingAlgorithmEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String algorithmName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AlterShardingAlgorithmEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AlterShardingAlgorithmEvent.java
new file mode 100644
index 00000000000..fb164079902
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/AlterShardingAlgorithmEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.algorithm.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter sharding algorithm event.
+ *
+ * @param <T> algorithm configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingAlgorithmEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String algorithmName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/DeleteShardingAlgorithmEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/DeleteShardingAlgorithmEvent.java
new file mode 100644
index 00000000000..ec6363a8eb0
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/algorithm/sharding/DeleteShardingAlgorithmEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.algorithm.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete sharding algorithm event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingAlgorithmEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String algorithmName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AddShardingCacheConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AddShardingCacheConfigurationEvent.java
new file mode 100644
index 00000000000..dfaeea32231
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AddShardingCacheConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.cache;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.cache.ShardingCacheConfiguration;
+
+/**
+ * Add sharding cache configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingCacheConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingCacheConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AlterShardingCacheConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AlterShardingCacheConfigurationEvent.java
new file mode 100644
index 00000000000..beaea70f09c
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/AlterShardingCacheConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.cache;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.cache.ShardingCacheConfiguration;
+
+/**
+ * Alter sharding cache configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingCacheConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingCacheConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/DeleteShardingCacheConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/DeleteShardingCacheConfigurationEvent.java
new file mode 100644
index 00000000000..9e0878bc652
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/cache/DeleteShardingCacheConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.cache;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete sharding cache configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingCacheConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AddShardingAuditorStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AddShardingAuditorStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..b27f665c1de
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AddShardingAuditorStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.audit;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
+
+/**
+ * Add default sharding auditor strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingAuditorStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingAuditStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AlterShardingAuditorStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AlterShardingAuditorStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..b3123e7797b
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/AlterShardingAuditorStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.audit;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
+
+/**
+ * Alter default sharding auditor strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingAuditorStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingAuditStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/DeleteShardingAuditorStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/DeleteShardingAuditorStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..8f30a81b869
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/audit/DeleteShardingAuditorStrategyConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.strategy.audit;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete default sharding auditor strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingAuditorStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AddDatabaseShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AddDatabaseShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..7b3bf1148bc
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AddDatabaseShardingStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.database;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+
+/**
+ * Add default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddDatabaseShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AlterDatabaseShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AlterDatabaseShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..eaed724e4ee
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/AlterDatabaseShardingStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.database;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+
+/**
+ * Alter default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterDatabaseShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/DeleteDatabaseShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/DeleteDatabaseShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..a6043d15ad9
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/database/DeleteDatabaseShardingStrategyConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.strategy.database;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteDatabaseShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AddKeyGenerateStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AddKeyGenerateStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..7fb4ac0b231
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AddKeyGenerateStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.keygenerate;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+
+/**
+ * Add default key generate strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddKeyGenerateStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final KeyGenerateStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AlterKeyGenerateStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AlterKeyGenerateStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..920ef33a744
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/AlterKeyGenerateStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.keygenerate;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+
+/**
+ * Alter default key generate strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterKeyGenerateStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final KeyGenerateStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/DeleteKeyGenerateStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/DeleteKeyGenerateStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..5ea261f2bdc
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/keygenerate/DeleteKeyGenerateStrategyConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.strategy.keygenerate;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete default key generate strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteKeyGenerateStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AddDefaultShardingColumnEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AddDefaultShardingColumnEvent.java
new file mode 100644
index 00000000000..1367b535c50
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AddDefaultShardingColumnEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.strategy.shardingcolumn;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add default sharding column event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddDefaultShardingColumnEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AlterDefaultShardingColumnEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AlterDefaultShardingColumnEvent.java
new file mode 100644
index 00000000000..f71cbfaafba
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/AlterDefaultShardingColumnEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.strategy.shardingcolumn;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter default sharding column event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterDefaultShardingColumnEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/DeleteDefaultShardingColumnEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/DeleteDefaultShardingColumnEvent.java
new file mode 100644
index 00000000000..5b68a6f910c
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/shardingcolumn/DeleteDefaultShardingColumnEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.strategy.shardingcolumn;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete default sharding column event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteDefaultShardingColumnEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AddTableShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AddTableShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..ec135c56f5d
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AddTableShardingStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.table;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+
+/**
+ * Add default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddTableShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AlterTableShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AlterTableShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..29750e304ab
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/AlterTableShardingStrategyConfigurationEvent.java
@@ -0,0 +1,35 @@
+/*
+ * 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.sharding.event.strategy.table;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+
+/**
+ * Alter default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterTableShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final ShardingStrategyConfiguration config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/DeleteTableShardingStrategyConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/DeleteTableShardingStrategyConfigurationEvent.java
new file mode 100644
index 00000000000..146b9445aeb
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/strategy/table/DeleteTableShardingStrategyConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.strategy.table;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete default database sharding strategy configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteTableShardingStrategyConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AddShardingAutoTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AddShardingAutoTableConfigurationEvent.java
new file mode 100644
index 00000000000..2e158285274
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AddShardingAutoTableConfigurationEvent.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.event.table.auto;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add sharding auto table configuration event.
+ *
+ * @param <T> sharding auto table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingAutoTableConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AlterShardingAutoTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AlterShardingAutoTableConfigurationEvent.java
new file mode 100644
index 00000000000..6b1593e789f
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/AlterShardingAutoTableConfigurationEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.table.auto;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter sharding auto table configuration event.
+ *
+ * @param <T> sharding auto table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingAutoTableConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/DeleteShardingAutoTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/DeleteShardingAutoTableConfigurationEvent.java
new file mode 100644
index 00000000000..b41ff63eb70
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/auto/DeleteShardingAutoTableConfigurationEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.table.auto;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete sharding auto table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingAutoTableConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AddShardingTableReferenceConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AddShardingTableReferenceConfigurationEvent.java
new file mode 100644
index 00000000000..fe6c7b2c00b
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AddShardingTableReferenceConfigurationEvent.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.event.table.binding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add sharding binding table configuration event.
+ *
+ * @param <T> sharding binding table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingTableReferenceConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AlterShardingTableReferenceConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AlterShardingTableReferenceConfigurationEvent.java
new file mode 100644
index 00000000000..b4f9e19f9d3
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/AlterShardingTableReferenceConfigurationEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.table.binding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter sharding binding table configuration event.
+ *
+ * @param <T> sharding binding table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingTableReferenceConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/DeleteShardingTableReferenceConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/DeleteShardingTableReferenceConfigurationEvent.java
new file mode 100644
index 00000000000..a90200956df
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/binding/DeleteShardingTableReferenceConfigurationEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.table.binding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete sharding binding table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingTableReferenceConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AddBroadcastTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AddBroadcastTableConfigurationEvent.java
new file mode 100644
index 00000000000..d91d41e4e9b
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AddBroadcastTableConfigurationEvent.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.event.table.broadcast;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+import java.util.Collection;
+
+/**
+ * Add broadcast table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddBroadcastTableConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final Collection<String> config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AlterBroadcastTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AlterBroadcastTableConfigurationEvent.java
new file mode 100644
index 00000000000..13a0fd4397a
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/AlterBroadcastTableConfigurationEvent.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.event.table.broadcast;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+import java.util.Collection;
+
+/**
+ * Alter broadcast table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterBroadcastTableConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final Collection<String> config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/DeleteBroadcastTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/DeleteBroadcastTableConfigurationEvent.java
new file mode 100644
index 00000000000..53d197cf0b7
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/broadcast/DeleteBroadcastTableConfigurationEvent.java
@@ -0,0 +1,32 @@
+/*
+ * 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.sharding.event.table.broadcast;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete broadcast table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteBroadcastTableConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AddShardingTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AddShardingTableConfigurationEvent.java
new file mode 100644
index 00000000000..ce0c1a7f31a
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AddShardingTableConfigurationEvent.java
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.sharding.event.table.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Add sharding table configuration event.
+ *
+ * @param <T> sharding table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AddShardingTableConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AlterShardingTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AlterShardingTableConfigurationEvent.java
new file mode 100644
index 00000000000..f60a0c44445
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/AlterShardingTableConfigurationEvent.java
@@ -0,0 +1,38 @@
+/*
+ * 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.sharding.event.table.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Alter sharding table configuration event.
+ *
+ * @param <T> sharding table configuration
+ */
+@RequiredArgsConstructor
+@Getter
+public final class AlterShardingTableConfigurationEvent<T> implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+    
+    private final T config;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/DeleteShardingTableConfigurationEvent.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/DeleteShardingTableConfigurationEvent.java
new file mode 100644
index 00000000000..53ed30de9af
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/event/table/sharding/DeleteShardingTableConfigurationEvent.java
@@ -0,0 +1,34 @@
+/*
+ * 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.sharding.event.table.sharding;
+
+import lombok.Getter;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.rule.event.GovernanceEvent;
+
+/**
+ * Delete sharding table configuration event.
+ */
+@RequiredArgsConstructor
+@Getter
+public final class DeleteShardingTableConfigurationEvent implements GovernanceEvent {
+    
+    private final String databaseName;
+    
+    private final String tableName;
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverter.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverter.java
index 91e3eea9bfc..59d82486a5a 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverter.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverter.java
@@ -242,7 +242,7 @@ public final class ShardingNodeConverter {
      * @return true or false
      */
     public static boolean isBroadcastTablePath(final String rulePath) {
-        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + BROADCAST_TABLES_NODE + "\\.*", Pattern.CASE_INSENSITIVE);
+        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + BROADCAST_TABLES_NODE + "$", Pattern.CASE_INSENSITIVE);
         Matcher matcher = pattern.matcher(rulePath);
         return matcher.find();
     }
@@ -391,18 +391,6 @@ public final class ShardingNodeConverter {
         return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
     }
     
-    /**
-     * Get broadcast name.
-     *
-     * @param rulePath rule path
-     * @return broadcast name
-     */
-    public static Optional<String> getBroadcastTableName(final String rulePath) {
-        Pattern pattern = Pattern.compile(RULES_NODE_PREFIX + ROOT_NODE + "/" + BROADCAST_TABLES_NODE + RULE_NAME_PATTERN, Pattern.CASE_INSENSITIVE);
-        Matcher matcher = pattern.matcher(rulePath);
-        return matcher.find() ? Optional.of(matcher.group(3)) : Optional.empty();
-    }
-    
     /**
      * Get sharding algorithm name.
      *
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingAlgorithmSubscriber.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingAlgorithmSubscriber.java
new file mode 100644
index 00000000000..116b9360d23
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingAlgorithmSubscriber.java
@@ -0,0 +1,199 @@
+/*
+ * 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.sharding.subscriber;
+
+import com.google.common.eventbus.Subscribe;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.algorithm.AlgorithmConfiguration;
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator;
+import org.apache.shardingsphere.mode.event.config.DatabaseRuleConfigurationChangedEvent;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.AddAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.AlterAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.auditor.DeleteAuditorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.AddKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.AlterKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.keygenerator.DeleteKeyGeneratorEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.AddShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.AlterShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.event.algorithm.sharding.DeleteShardingAlgorithmEvent;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * Sharding algorithm subscriber.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@RequiredArgsConstructor
+public final class ShardingAlgorithmSubscriber implements RuleConfigurationSubscribeCoordinator {
+    
+    private Map<String, ShardingSphereDatabase> databases;
+    
+    private InstanceContext instanceContext;
+    
+    @Override
+    public void registerRuleConfigurationSubscriber(final Map<String, ShardingSphereDatabase> databases, final InstanceContext instanceContext) {
+        this.databases = databases;
+        this.instanceContext = instanceContext;
+        instanceContext.getEventBusContext().register(this);
+    }
+    
+    /**
+     * Renew with add sharding algorithm.
+     *
+     * @param event add sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingAlgorithmEvent<AlgorithmConfiguration> event) {
+        renewShardingAlgorithm(event.getDatabaseName(), event.getAlgorithmName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with add sharding algorithm.
+     *
+     * @param event add sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AddKeyGeneratorEvent<AlgorithmConfiguration> event) {
+        renewKeyGenerator(event.getDatabaseName(), event.getKeyGeneratorName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with add sharding algorithm.
+     *
+     * @param event add sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AddAuditorEvent<AlgorithmConfiguration> event) {
+        renewAuditor(event.getDatabaseName(), event.getAuditorName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with alter sharding algorithm.
+     *
+     * @param event alter sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingAlgorithmEvent<AlgorithmConfiguration> event) {
+        renewShardingAlgorithm(event.getDatabaseName(), event.getAlgorithmName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with alter sharding algorithm.
+     *
+     * @param event alter sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterKeyGeneratorEvent<AlgorithmConfiguration> event) {
+        renewKeyGenerator(event.getDatabaseName(), event.getKeyGeneratorName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with alter sharding algorithm.
+     *
+     * @param event alter sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterAuditorEvent<AlgorithmConfiguration> event) {
+        renewAuditor(event.getDatabaseName(), event.getAuditorName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with delete sharding algorithm.
+     *
+     * @param event delete sharding algorithm event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingAlgorithmEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getShardingAlgorithms().remove(event.getAlgorithmName());
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete key generator.
+     *
+     * @param event delete key generator event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteKeyGeneratorEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getKeyGenerators().remove(event.getKeyGeneratorName());
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete key generator.
+     *
+     * @param event delete key generator event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteAuditorEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getAuditors().remove(event.getAuditorName());
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    private void renewShardingAlgorithm(final String databaseName, final String algorithmName, final AlgorithmConfiguration algorithmConfig) {
+        ShardingSphereDatabase database = databases.get(databaseName);
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getShardingAlgorithms().put(algorithmName, algorithmConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(databaseName, config));
+    }
+    
+    private void renewKeyGenerator(final String databaseName, final String algorithmName, final AlgorithmConfiguration algorithmConfig) {
+        ShardingSphereDatabase database = databases.get(databaseName);
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getKeyGenerators().put(algorithmName, algorithmConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(databaseName, config));
+    }
+    
+    private void renewAuditor(final String databaseName, final String algorithmName, final AlgorithmConfiguration algorithmConfig) {
+        ShardingSphereDatabase database = databases.get(databaseName);
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getAuditors().put(algorithmName, algorithmConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(databaseName, config));
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingCacheConfigurationSubscriber.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingCacheConfigurationSubscriber.java
new file mode 100644
index 00000000000..e6c3fe4aecf
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingCacheConfigurationSubscriber.java
@@ -0,0 +1,101 @@
+/*
+ * 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.sharding.subscriber;
+
+import com.google.common.eventbus.Subscribe;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator;
+import org.apache.shardingsphere.mode.event.config.DatabaseRuleConfigurationChangedEvent;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.cache.ShardingCacheConfiguration;
+import org.apache.shardingsphere.sharding.event.cache.AddShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.cache.AlterShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.cache.DeleteShardingCacheConfigurationEvent;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * Sharding cache configuration subscriber.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@RequiredArgsConstructor
+public final class ShardingCacheConfigurationSubscriber implements RuleConfigurationSubscribeCoordinator {
+    
+    private Map<String, ShardingSphereDatabase> databases;
+    
+    private InstanceContext instanceContext;
+    
+    @Override
+    public void registerRuleConfigurationSubscriber(final Map<String, ShardingSphereDatabase> databases, final InstanceContext instanceContext) {
+        this.databases = databases;
+        this.instanceContext = instanceContext;
+        instanceContext.getEventBusContext().register(this);
+    }
+    
+    /**
+     * Renew with add sharding cache configuration.
+     *
+     * @param event add sharding cache configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingCacheConfigurationEvent event) {
+        renewShardingCacheConfig(event.getDatabaseName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with alter sharding cache configuration.
+     *
+     * @param event alter sharding cache configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingCacheConfigurationEvent event) {
+        renewShardingCacheConfig(event.getDatabaseName(), event.getConfig());
+    }
+    
+    /**
+     * Renew with delete sharding cache configuration.
+     *
+     * @param event delete sharding cache configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingCacheConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setShardingCache(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    private void renewShardingCacheConfig(final String databaseName, final ShardingCacheConfiguration shardingCacheConfiguration) {
+        ShardingSphereDatabase database = databases.get(databaseName);
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setShardingCache(shardingCacheConfiguration);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(databaseName, config));
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingStrategyConfigurationSubscriber.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingStrategyConfigurationSubscriber.java
new file mode 100644
index 00000000000..24c9d8e2046
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingStrategyConfigurationSubscriber.java
@@ -0,0 +1,355 @@
+/*
+ * 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.sharding.subscriber;
+
+import com.google.common.eventbus.Subscribe;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator;
+import org.apache.shardingsphere.mode.event.config.DatabaseRuleConfigurationChangedEvent;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.audit.ShardingAuditStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.keygen.KeyGenerateStrategyConfiguration;
+import org.apache.shardingsphere.sharding.api.config.strategy.sharding.ShardingStrategyConfiguration;
+import org.apache.shardingsphere.sharding.event.strategy.audit.AddShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.audit.AlterShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.audit.DeleteShardingAuditorStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.AddDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.AlterDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.database.DeleteDatabaseShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.AddKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.AlterKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.keygenerate.DeleteKeyGenerateStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.AddDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.AlterDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.shardingcolumn.DeleteDefaultShardingColumnEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.AddTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.AlterTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.strategy.table.DeleteTableShardingStrategyConfigurationEvent;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding strategy configuration subscriber.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@RequiredArgsConstructor
+public final class ShardingStrategyConfigurationSubscriber implements RuleConfigurationSubscribeCoordinator {
+    
+    private Map<String, ShardingSphereDatabase> databases;
+    
+    private InstanceContext instanceContext;
+    
+    @Override
+    public void registerRuleConfigurationSubscriber(final Map<String, ShardingSphereDatabase> databases, final InstanceContext instanceContext) {
+        this.databases = databases;
+        this.instanceContext = instanceContext;
+        instanceContext.getEventBusContext().register(this);
+    }
+    
+    /**
+     * Renew with add default database sharding strategy configuration.
+     *
+     * @param event add default database sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddDatabaseShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingStrategyConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setDefaultDatabaseShardingStrategy(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.setDefaultDatabaseShardingStrategy(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add default table sharding strategy configuration.
+     *
+     * @param event add default table sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddTableShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingStrategyConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setDefaultTableShardingStrategy(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.setDefaultTableShardingStrategy(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add default key generate strategy configuration.
+     *
+     * @param event add default key generate strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddKeyGenerateStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        KeyGenerateStrategyConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setDefaultKeyGenerateStrategy(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.setDefaultKeyGenerateStrategy(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add default sharding auditor strategy configuration.
+     *
+     * @param event add default sharding auditor strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingAuditorStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingAuditStrategyConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setDefaultAuditStrategy(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.setDefaultAuditStrategy(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add default sharding column configuration.
+     *
+     * @param event add default sharding column configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddDefaultShardingColumnEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        String needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setDefaultShardingColumn(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.setDefaultShardingColumn(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter default database sharding strategy configuration.
+     *
+     * @param event alter default database sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterDatabaseShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingStrategyConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultDatabaseShardingStrategy(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter default table sharding strategy configuration.
+     *
+     * @param event alter default table sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterTableShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingStrategyConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultTableShardingStrategy(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter default key generate strategy configuration.
+     *
+     * @param event alter default key generate strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterKeyGenerateStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        KeyGenerateStrategyConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultKeyGenerateStrategy(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter default sharding auditor strategy configuration.
+     *
+     * @param event alter default sharding auditor strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingAuditorStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingAuditStrategyConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultAuditStrategy(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter default sharding column configuration.
+     *
+     * @param event alter default sharding column configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterDefaultShardingColumnEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        String needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultShardingColumn(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete default database sharding strategy configuration.
+     *
+     * @param event delete default database sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteDatabaseShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultDatabaseShardingStrategy(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete default table sharding strategy configuration.
+     *
+     * @param event delete default table sharding strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteTableShardingStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultTableShardingStrategy(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete default key generate strategy configuration.
+     *
+     * @param event delete default key generate strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteKeyGenerateStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultKeyGenerateStrategy(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete default sharding auditor strategy configuration.
+     *
+     * @param event delete default sharding auditor strategy configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingAuditorStrategyConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultAuditStrategy(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete default sharding column configuration.
+     *
+     * @param event delete default sharding column configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteDefaultShardingColumnEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setDefaultAuditStrategy(null);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingTableConfigurationSubscriber.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingTableConfigurationSubscriber.java
new file mode 100644
index 00000000000..91ed4357e9c
--- /dev/null
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/subscriber/ShardingTableConfigurationSubscriber.java
@@ -0,0 +1,299 @@
+/*
+ * 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.sharding.subscriber;
+
+import com.google.common.eventbus.Subscribe;
+import lombok.RequiredArgsConstructor;
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.instance.InstanceContext;
+import org.apache.shardingsphere.infra.metadata.database.ShardingSphereDatabase;
+import org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator;
+import org.apache.shardingsphere.mode.event.config.DatabaseRuleConfigurationChangedEvent;
+import org.apache.shardingsphere.sharding.api.config.ShardingRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingAutoTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableReferenceRuleConfiguration;
+import org.apache.shardingsphere.sharding.api.config.rule.ShardingTableRuleConfiguration;
+import org.apache.shardingsphere.sharding.event.table.auto.AddShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.auto.AlterShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.auto.DeleteShardingAutoTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.binding.AddShardingTableReferenceConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.binding.AlterShardingTableReferenceConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.binding.DeleteShardingTableReferenceConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.AddBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.AlterBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.broadcast.DeleteBroadcastTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.AddShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.AlterShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.event.table.sharding.DeleteShardingTableConfigurationEvent;
+import org.apache.shardingsphere.sharding.rule.ShardingRule;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+import java.util.Optional;
+
+/**
+ * Sharding table configuration subscriber.
+ */
+@SuppressWarnings("UnstableApiUsage")
+@RequiredArgsConstructor
+public final class ShardingTableConfigurationSubscriber implements RuleConfigurationSubscribeCoordinator {
+    
+    private Map<String, ShardingSphereDatabase> databases;
+    
+    private InstanceContext instanceContext;
+    
+    @Override
+    public void registerRuleConfigurationSubscriber(final Map<String, ShardingSphereDatabase> databases, final InstanceContext instanceContext) {
+        this.databases = databases;
+        this.instanceContext = instanceContext;
+        instanceContext.getEventBusContext().register(this);
+    }
+    
+    /**
+     * Renew with add sharding table configuration.
+     *
+     * @param event add sharding table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingTableConfigurationEvent<ShardingTableRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingTableRuleConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.getTables().add(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.getTables().add(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add sharding auto table configuration.
+     *
+     * @param event add sharding auto table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingAutoTableConfigurationEvent<ShardingAutoTableRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingAutoTableRuleConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.getAutoTables().add(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.getAutoTables().add(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add sharding auto table configuration.
+     *
+     * @param event add sharding auto table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddShardingTableReferenceConfigurationEvent<ShardingTableReferenceRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingTableReferenceRuleConfiguration needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.getBindingTableGroups().add(needToAddedConfig);
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.getBindingTableGroups().add(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with add broadcast table configuration.
+     *
+     * @param event add broadcast table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AddBroadcastTableConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<String> needToAddedConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        Optional<ShardingRule> rule = database.getRuleMetaData().findSingleRule(ShardingRule.class);
+        ShardingRuleConfiguration config;
+        if (rule.isPresent()) {
+            config = (ShardingRuleConfiguration) rule.get().getConfiguration();
+            config.setBroadcastTables(new LinkedList<>(needToAddedConfig));
+        } else {
+            config = new ShardingRuleConfiguration();
+            config.getBroadcastTables().addAll(needToAddedConfig);
+        }
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter sharding table configuration.
+     *
+     * @param event alter sharding table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingTableConfigurationEvent<ShardingTableRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingTableRuleConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getTables().removeIf(each -> each.getLogicTable().equals(event.getTableName()));
+        config.getTables().add(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter sharding auto table configuration.
+     *
+     * @param event alter sharding auto table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingAutoTableConfigurationEvent<ShardingAutoTableRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingAutoTableRuleConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getAutoTables().removeIf(each -> each.getLogicTable().equals(event.getTableName()));
+        config.getAutoTables().add(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter sharding table reference configuration.
+     *
+     * @param event alter sharding table reference configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterShardingTableReferenceConfigurationEvent<ShardingTableReferenceRuleConfiguration> event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        ShardingTableReferenceRuleConfiguration needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getBindingTableGroups().removeIf(each -> each.getName().equals(event.getTableName()));
+        config.getBindingTableGroups().add(needToAlteredConfig);
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with alter broadcast table configuration.
+     *
+     * @param event alter broadcast table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final AlterBroadcastTableConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<String> needToAlteredConfig = event.getConfig();
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setBroadcastTables(new LinkedList<>(needToAlteredConfig));
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete sharding table configuration.
+     *
+     * @param event delete sharding table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingTableConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getTables().removeIf(each -> each.getLogicTable().equals(event.getTableName()));
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete sharding auto table configuration.
+     *
+     * @param event delete sharding auto table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingAutoTableConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getAutoTables().removeIf(each -> each.getLogicTable().equals(event.getTableName()));
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete sharding table reference configuration.
+     *
+     * @param event delete sharding table reference configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteShardingTableReferenceConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.getBindingTableGroups().removeIf(each -> each.getName().equals(event.getTableName()));
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+    
+    /**
+     * Renew with delete broadcast table configuration.
+     *
+     * @param event delete broadcast table configuration event
+     */
+    @Subscribe
+    public synchronized void renew(final DeleteBroadcastTableConfigurationEvent event) {
+        ShardingSphereDatabase database = databases.get(event.getDatabaseName());
+        Collection<RuleConfiguration> ruleConfigs = new LinkedList<>(database.getRuleMetaData().getConfigurations());
+        ShardingRuleConfiguration config = (ShardingRuleConfiguration) database.getRuleMetaData().getSingleRule(ShardingRule.class).getConfiguration();
+        config.setBroadcastTables(new ArrayList<>());
+        ruleConfigs.add(config);
+        database.getRuleMetaData().getConfigurations().addAll(ruleConfigs);
+        instanceContext.getEventBusContext().post(new DatabaseRuleConfigurationChangedEvent(event.getDatabaseName(), config));
+    }
+}
diff --git a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapper.java b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapper.java
index b24ef7242b5..9aebded517b 100644
--- a/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapper.java
+++ b/features/sharding/core/src/main/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapper.java
@@ -142,7 +142,7 @@ public final class NewYamlShardingRuleConfigurationSwapper implements NewYamlRul
                 ShardingNodeConverter.getBindingTableName(each.getKey())
                         .ifPresent(bindingTableName -> result.getBindingTableGroups().add(YamlShardingTableReferenceRuleConfigurationConverter.convertToObject(each.getValue())));
             } else if (ShardingNodeConverter.isBroadcastTablePath(each.getKey())) {
-                ShardingNodeConverter.getBroadcastTableName(each.getKey()).ifPresent(broadcastName -> result.getBroadcastTables().add(each.getValue()));
+                result.getBroadcastTables().addAll(YamlEngine.unmarshal(each.getValue(), Collection.class));
             } else if (ShardingNodeConverter.isDefaultDatabaseStrategyPath(each.getKey())) {
                 result.setDefaultDatabaseShardingStrategy(shardingStrategySwapper.swapToObject(YamlEngine.unmarshal(each.getValue(), YamlShardingStrategyConfiguration.class)));
             } else if (ShardingNodeConverter.isDefaultTableStrategyPath(each.getKey())) {
diff --git a/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator b/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator
new file mode 100644
index 00000000000..9f6800508a5
--- /dev/null
+++ b/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.rule.RuleConfigurationSubscribeCoordinator
@@ -0,0 +1,21 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.sharding.subscriber.ShardingTableConfigurationSubscriber
+org.apache.shardingsphere.sharding.subscriber.ShardingStrategyConfigurationSubscriber
+org.apache.shardingsphere.sharding.subscriber.ShardingAlgorithmSubscriber
+org.apache.shardingsphere.sharding.subscriber.ShardingCacheConfigurationSubscriber
diff --git a/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mode.event.config.RuleConfigurationEventBuilder b/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mode.event.config.RuleConfigurationEventBuilder
new file mode 100644
index 00000000000..a39faee1432
--- /dev/null
+++ b/features/sharding/core/src/main/resources/META-INF/services/org.apache.shardingsphere.mode.event.config.RuleConfigurationEventBuilder
@@ -0,0 +1,18 @@
+#
+# 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.
+#
+
+org.apache.shardingsphere.sharding.event.ShardingRuleConfigurationEventBuilder
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverterTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverterTest.java
index 7761d70a034..266e34f9a46 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverterTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/metadata/converter/ShardingNodeConverterTest.java
@@ -103,8 +103,8 @@ class ShardingNodeConverterTest {
         assertFalse(ShardingNodeConverter.isAutoTablePath("/metadata/foo_db/rules/sharding/algorithms/MD5"));
         assertTrue(ShardingNodeConverter.isBindingTablePath("/metadata/foo_db/rules/sharding/binding_tables/foo_table"));
         assertFalse(ShardingNodeConverter.isBindingTablePath("/metadata/foo_db/rules/sharding/algorithms/MD5"));
-        assertTrue(ShardingNodeConverter.isBroadcastTablePath("/metadata/foo_db/rules/sharding/broadcast_tables/foo_table"));
-        assertFalse(ShardingNodeConverter.isBroadcastTablePath("/metadata/foo_db/rules/sharding/algorithms/MD5"));
+        assertTrue(ShardingNodeConverter.isBroadcastTablePath("/metadata/foo_db/rules/sharding/broadcast_tables"));
+        assertFalse(ShardingNodeConverter.isBroadcastTablePath("/metadata/foo_db/rules/sharding/broadcast_tables/foo"));
         assertTrue(ShardingNodeConverter.isDefaultDatabaseStrategyPath("/metadata/foo_db/rules/sharding/default_strategies/default_database_strategy"));
         assertFalse(ShardingNodeConverter.isDefaultDatabaseStrategyPath("/metadata/foo_db/rules/sharding/default_strategies/default_database_strategy/foo"));
         assertTrue(ShardingNodeConverter.isDefaultTableStrategyPath("/metadata/foo_db/rules/sharding/default_strategies/default_table_strategy"));
@@ -146,13 +146,6 @@ class ShardingNodeConverterTest {
         assertThat(actual.get(), is("foo_table"));
     }
     
-    @Test
-    void assertGetBroadcastTableNameByRulePath() {
-        Optional<String> actual = ShardingNodeConverter.getBroadcastTableName("/metadata/foo_db/rules/sharding/broadcast_tables/foo_table");
-        assertTrue(actual.isPresent());
-        assertThat(actual.get(), is("foo_table"));
-    }
-    
     @Test
     void assertGetAlgorithmNameByRulePath() {
         Optional<String> actual = ShardingNodeConverter.getShardingAlgorithmName("/metadata/foo_db/rules/sharding/algorithms/foo");
diff --git a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapperTest.java b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapperTest.java
index 52531978e9f..0c0617b6645 100644
--- a/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapperTest.java
+++ b/features/sharding/core/src/test/java/org/apache/shardingsphere/sharding/yaml/swapper/NewYamlShardingRuleConfigurationSwapperTest.java
@@ -92,6 +92,7 @@ class NewYamlShardingRuleConfigurationSwapperTest {
         result.getAutoTables().add(autoTableRuleConfiguration);
         result.getBindingTableGroups().add(new ShardingTableReferenceRuleConfiguration("foo", shardingTableRuleConfig.getLogicTable() + "," + subTableRuleConfig.getLogicTable()));
         result.getBroadcastTables().add("BROADCAST_TABLE");
+        result.getBroadcastTables().add("BROADCAST_TABLE_SUB");
         result.setDefaultDatabaseShardingStrategy(new StandardShardingStrategyConfiguration("ds_id", "standard"));
         result.setDefaultTableShardingStrategy(new StandardShardingStrategyConfiguration("table_id", "standard"));
         result.setDefaultShardingColumn("table_id");
@@ -179,7 +180,8 @@ class NewYamlShardingRuleConfigurationSwapperTest {
                 + "    shardingAlgorithmName: hash_mod\n"
                 + "    shardingColumn: user_id\n"));
         config.add(new YamlDataNode("/metadata/foo_db/rules/sharding/binding_tables/foo", "foo:LOGIC_TABLE,SUB_LOGIC_TABLE"));
-        config.add(new YamlDataNode("/metadata/foo_db/rules/sharding/broadcast_tables", "- BROADCAST_TABLE\n"));
+        config.add(new YamlDataNode("/metadata/foo_db/rules/sharding/broadcast_tables", "- BROADCAST_TABLE\n"
+                + "- BROADCAST_TABLE_SUB\n"));
         config.add(new YamlDataNode("/metadata/foo_db/rules/sharding/default_strategies/default_database_strategy", "standard:\n"
                 + "  shardingAlgorithmName: standard\n"
                 + "  shardingColumn: ds_id\n"));
@@ -232,6 +234,10 @@ class NewYamlShardingRuleConfigurationSwapperTest {
         assertThat(result.getBindingTableGroups().size(), is(1));
         assertThat(result.getBindingTableGroups().iterator().next().getName(), is("foo"));
         assertThat(result.getBindingTableGroups().iterator().next().getReference(), is("LOGIC_TABLE,SUB_LOGIC_TABLE"));
+        assertThat(result.getBroadcastTables().size(), is(2));
+        Iterator<String> broadcastIterator = result.getBroadcastTables().iterator();
+        assertThat(broadcastIterator.next(), is("BROADCAST_TABLE"));
+        assertThat(broadcastIterator.next(), is("BROADCAST_TABLE_SUB"));
         assertTrue(result.getDefaultDatabaseShardingStrategy() instanceof StandardShardingStrategyConfiguration);
         assertThat(((StandardShardingStrategyConfiguration) result.getDefaultDatabaseShardingStrategy()).getType(), is("STANDARD"));
         assertThat(((StandardShardingStrategyConfiguration) result.getDefaultDatabaseShardingStrategy()).getShardingColumn(), is("ds_id"));