You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@shardingsphere.apache.org by du...@apache.org on 2023/06/21 13:34:49 UTC

[shardingsphere] branch master updated: Differentiate between global and database rule swapper (#26474)

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

duanzhengqiang 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 3f3d0738302 Differentiate between global and database rule swapper (#26474)
3f3d0738302 is described below

commit 3f3d0738302ea9c11fa0251e7b4484c53586676e
Author: zhaojinchao <zh...@apache.org>
AuthorDate: Wed Jun 21 21:34:42 2023 +0800

    Differentiate between global and database rule swapper (#26474)
    
    * Differentiate between global and database rule swapper
    
    * Update comment
    
    * Update
    
    * Modify spi
---
 .../rule/NewYamGlobalRuleConfigurationSwapper.java | 40 +++++++++++++++
 ...ewYamlGlobalRuleConfigurationSwapperEngine.java | 59 ++++++++++++++++++++++
 .../NewYamlAuthorityRuleConfigurationSwapper.java  |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 ...NewYamlGlobalClockRuleConfigurationSwapper.java |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 .../NewYamlLoggingRuleConfigurationSwapper.java    |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 .../config/global/NewGlobalRulePersistService.java |  8 +--
 ...wYamlSQLFederationRuleConfigurationSwapper.java |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 .../NewYamlSQLParserRuleConfigurationSwapper.java  |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 ...wYamlSQLTranslatorRuleConfigurationSwapper.java |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 .../NewYamlTrafficRuleConfigurationSwapper.java    |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 ...NewYamlTransactionRuleConfigurationSwapper.java |  4 +-
 ...pper.rule.NewYamGlobalRuleConfigurationSwapper} |  0
 19 files changed, 120 insertions(+), 19 deletions(-)

diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamGlobalRuleConfigurationSwapper.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamGlobalRuleConfigurationSwapper.java
new file mode 100644
index 00000000000..98ea2230228
--- /dev/null
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamGlobalRuleConfigurationSwapper.java
@@ -0,0 +1,40 @@
+/*
+ * 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.infra.yaml.config.swapper.rule;
+
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.util.spi.annotation.SingletonSPI;
+import org.apache.shardingsphere.infra.util.spi.type.ordered.OrderedSPI;
+import org.apache.shardingsphere.infra.util.yaml.swapper.NewYamlConfigurationSwapper;
+
+/**
+ * TODO Rename YamGlobalRuleConfigurationSwapper when metadata structure adjustment completed. #25485
+ * YAML rule global configuration swapper.
+ *
+ * @param <T> type of rule configuration
+ */
+@SingletonSPI
+public interface NewYamGlobalRuleConfigurationSwapper<T extends RuleConfiguration> extends NewYamlConfigurationSwapper<T>, OrderedSPI<T> {
+    
+    /**
+     * Get YAML rule tag name.
+     * 
+     * @return YAML rule tag name
+     */
+    String getRuleTagName();
+}
diff --git a/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamlGlobalRuleConfigurationSwapperEngine.java b/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamlGlobalRuleConfigurationSwapperEngine.java
new file mode 100644
index 00000000000..9660ee0e1e0
--- /dev/null
+++ b/infra/common/src/main/java/org/apache/shardingsphere/infra/yaml/config/swapper/rule/NewYamlGlobalRuleConfigurationSwapperEngine.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.shardingsphere.infra.yaml.config.swapper.rule;
+
+import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
+import org.apache.shardingsphere.infra.util.spi.type.ordered.OrderedSPILoader;
+import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
+
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.Map;
+
+/**
+ * TODO Rename YamlGlobalRuleConfigurationSwapperEngine when metadata structure adjustment completed. #25485
+ * YAML global rule configuration swapper engine.
+ */
+public final class NewYamlGlobalRuleConfigurationSwapperEngine {
+    
+    /**
+     * Swap to YAML global rule configurations.
+     *
+     * @param ruleConfigs rule configurations
+     * @return YAML global rule configurations
+     */
+    @SuppressWarnings("rawtypes")
+    public Map<RuleConfiguration, NewYamGlobalRuleConfigurationSwapper> swapToYamlRuleConfigurations(final Collection<RuleConfiguration> ruleConfigs) {
+        return OrderedSPILoader.getServices(NewYamGlobalRuleConfigurationSwapper.class, ruleConfigs);
+    }
+    
+    /**
+     * Swap from YAML global rule configurations to rule configurations.
+     *
+     * @param dataNodes YAML data nodes
+     * @return global rule configurations
+     */
+    @SuppressWarnings({"unchecked", "rawtypes"})
+    public Collection<RuleConfiguration> swapToRuleConfigurations(final Collection<YamlDataNode> dataNodes) {
+        Collection<RuleConfiguration> result = new LinkedList<>();
+        for (NewYamGlobalRuleConfigurationSwapper each : OrderedSPILoader.getServices(NewYamGlobalRuleConfigurationSwapper.class)) {
+            result.add((RuleConfiguration) each.swapToObject(dataNodes));
+        }
+        return result;
+    }
+}
diff --git a/kernel/authority/core/src/main/java/org/apache/shardingsphere/authority/yaml/swapper/NewYamlAuthorityRuleConfigurationSwapper.java b/kernel/authority/core/src/main/java/org/apache/shardingsphere/authority/yaml/swapper/NewYamlAuthorityRuleConfigurationSwapper.java
index 1ca57cc9585..c0f521a7332 100644
--- a/kernel/authority/core/src/main/java/org/apache/shardingsphere/authority/yaml/swapper/NewYamlAuthorityRuleConfigurationSwapper.java
+++ b/kernel/authority/core/src/main/java/org/apache/shardingsphere/authority/yaml/swapper/NewYamlAuthorityRuleConfigurationSwapper.java
@@ -28,7 +28,7 @@ import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
 import org.apache.shardingsphere.infra.yaml.config.swapper.algorithm.YamlAlgorithmConfigurationSwapper;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -38,7 +38,7 @@ import java.util.Optional;
  * TODO Rename YamlAuthorityRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * New YAML Authority rule configuration swapper.
  */
-public final class NewYamlAuthorityRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<AuthorityRuleConfiguration> {
+public final class NewYamlAuthorityRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<AuthorityRuleConfiguration> {
     
     private final YamlAlgorithmConfigurationSwapper algorithmSwapper = new YamlAlgorithmConfigurationSwapper();
     
diff --git a/kernel/authority/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/authority/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/authority/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/authority/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/global-clock/core/src/main/java/org/apache/shardingsphere/globalclock/core/yaml/swapper/NewYamlGlobalClockRuleConfigurationSwapper.java b/kernel/global-clock/core/src/main/java/org/apache/shardingsphere/globalclock/core/yaml/swapper/NewYamlGlobalClockRuleConfigurationSwapper.java
index ff2607cbbd5..92b4861062c 100644
--- a/kernel/global-clock/core/src/main/java/org/apache/shardingsphere/globalclock/core/yaml/swapper/NewYamlGlobalClockRuleConfigurationSwapper.java
+++ b/kernel/global-clock/core/src/main/java/org/apache/shardingsphere/globalclock/core/yaml/swapper/NewYamlGlobalClockRuleConfigurationSwapper.java
@@ -23,7 +23,7 @@ import org.apache.shardingsphere.globalclock.core.yaml.config.YamlGlobalClockRul
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 
 import java.util.Collection;
 import java.util.Collections;
@@ -34,7 +34,7 @@ import java.util.Properties;
  * TODO Rename YamlGlobalClockRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * YAML global clock rule configuration swapper.
  */
-public final class NewYamlGlobalClockRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<GlobalClockRuleConfiguration> {
+public final class NewYamlGlobalClockRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<GlobalClockRuleConfiguration> {
     
     @Override
     public Collection<YamlDataNode> swapToDataNodes(final GlobalClockRuleConfiguration data) {
diff --git a/kernel/global-clock/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/global-clock/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/global-clock/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/global-clock/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/logging/core/src/main/java/org/apache/shardingsphere/logging/yaml/swapper/NewYamlLoggingRuleConfigurationSwapper.java b/kernel/logging/core/src/main/java/org/apache/shardingsphere/logging/yaml/swapper/NewYamlLoggingRuleConfigurationSwapper.java
index 6fecf838eb7..91cce84760f 100644
--- a/kernel/logging/core/src/main/java/org/apache/shardingsphere/logging/yaml/swapper/NewYamlLoggingRuleConfigurationSwapper.java
+++ b/kernel/logging/core/src/main/java/org/apache/shardingsphere/logging/yaml/swapper/NewYamlLoggingRuleConfigurationSwapper.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.logging.yaml.swapper;
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.logging.config.LoggingRuleConfiguration;
 import org.apache.shardingsphere.logging.constant.LoggingOrder;
 import org.apache.shardingsphere.logging.rule.builder.DefaultLoggingRuleConfigurationBuilder;
@@ -36,7 +36,7 @@ import java.util.Optional;
  * TODO Rename YamlLoggingRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * YAML logging rule configuration swapper.
  */
-public final class NewYamlLoggingRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<LoggingRuleConfiguration> {
+public final class NewYamlLoggingRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<LoggingRuleConfiguration> {
     
     @Override
     public Collection<YamlDataNode> swapToDataNodes(final LoggingRuleConfiguration data) {
diff --git a/kernel/logging/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/logging/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/logging/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/logging/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/metadata/core/src/main/java/org/apache/shardingsphere/metadata/persist/service/config/global/NewGlobalRulePersistService.java b/kernel/metadata/core/src/main/java/org/apache/shardingsphere/metadata/persist/service/config/global/NewGlobalRulePersistService.java
index 2e40107fbcd..2c2627fc12a 100644
--- a/kernel/metadata/core/src/main/java/org/apache/shardingsphere/metadata/persist/service/config/global/NewGlobalRulePersistService.java
+++ b/kernel/metadata/core/src/main/java/org/apache/shardingsphere/metadata/persist/service/config/global/NewGlobalRulePersistService.java
@@ -23,6 +23,8 @@ import org.apache.shardingsphere.infra.config.rule.RuleConfiguration;
 import org.apache.shardingsphere.infra.metadata.user.ShardingSphereUser;
 import org.apache.shardingsphere.infra.metadata.version.MetaDataVersion;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlGlobalRuleConfigurationSwapperEngine;
 import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
 import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapperEngine;
 import org.apache.shardingsphere.metadata.persist.node.NewGlobalNode;
@@ -55,8 +57,8 @@ public final class NewGlobalRulePersistService extends AbstractPersistService im
     @SuppressWarnings({"unchecked", "rawtypes"})
     @Override
     public void persist(final Collection<RuleConfiguration> globalRuleConfigs) {
-        Map<RuleConfiguration, NewYamlRuleConfigurationSwapper> yamlConfigs = new NewYamlRuleConfigurationSwapperEngine().swapToYamlRuleConfigurations(globalRuleConfigs);
-        for (Entry<RuleConfiguration, NewYamlRuleConfigurationSwapper> entry : yamlConfigs.entrySet()) {
+        Map<RuleConfiguration, NewYamGlobalRuleConfigurationSwapper> yamlConfigs = new NewYamlGlobalRuleConfigurationSwapperEngine().swapToYamlRuleConfigurations(globalRuleConfigs);
+        for (Entry<RuleConfiguration, NewYamGlobalRuleConfigurationSwapper> entry : yamlConfigs.entrySet()) {
             Collection<YamlDataNode> dataNodes = entry.getValue().swapToDataNodes(entry.getKey());
             if (dataNodes.isEmpty()) {
                 continue;
@@ -98,7 +100,7 @@ public final class NewGlobalRulePersistService extends AbstractPersistService im
     @Override
     public Collection<RuleConfiguration> load() {
         Collection<YamlDataNode> dataNodes = getDataNodes(NewGlobalNode.getGlobalRuleRootNode());
-        return dataNodes.isEmpty() ? Collections.emptyList() : new NewYamlRuleConfigurationSwapperEngine().swapToRuleConfigurations(dataNodes);
+        return dataNodes.isEmpty() ? Collections.emptyList() : new NewYamlGlobalRuleConfigurationSwapperEngine().swapToRuleConfigurations(dataNodes);
     }
     
     @Override
diff --git a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/NewYamlSQLFederationRuleConfigurationSwapper.java b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/NewYamlSQLFederationRuleConfigurationSwapper.java
index 01c4136db22..4f10b140bc3 100644
--- a/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/NewYamlSQLFederationRuleConfigurationSwapper.java
+++ b/kernel/sql-federation/core/src/main/java/org/apache/shardingsphere/sqlfederation/yaml/swapper/NewYamlSQLFederationRuleConfigurationSwapper.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sqlfederation.yaml.swapper;
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.sql.parser.api.CacheOption;
 import org.apache.shardingsphere.sqlfederation.api.config.SQLFederationRuleConfiguration;
 import org.apache.shardingsphere.sqlfederation.constant.SQLFederationOrder;
@@ -34,7 +34,7 @@ import java.util.Optional;
  * TODO Rename YamlSQLTranslatorRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * YAML SQL federation rule configuration swapper.
  */
-public final class NewYamlSQLFederationRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<SQLFederationRuleConfiguration> {
+public final class NewYamlSQLFederationRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<SQLFederationRuleConfiguration> {
     
     private final YamlSQLFederationExecutionPlanCacheConfigurationSwapper executionPlanCacheConfigSwapper = new YamlSQLFederationExecutionPlanCacheConfigurationSwapper();
     
diff --git a/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/sql-federation/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/sql-parser/core/src/main/java/org/apache/shardingsphere/parser/yaml/swapper/NewYamlSQLParserRuleConfigurationSwapper.java b/kernel/sql-parser/core/src/main/java/org/apache/shardingsphere/parser/yaml/swapper/NewYamlSQLParserRuleConfigurationSwapper.java
index 8909f1b38a0..097be60d265 100644
--- a/kernel/sql-parser/core/src/main/java/org/apache/shardingsphere/parser/yaml/swapper/NewYamlSQLParserRuleConfigurationSwapper.java
+++ b/kernel/sql-parser/core/src/main/java/org/apache/shardingsphere/parser/yaml/swapper/NewYamlSQLParserRuleConfigurationSwapper.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.parser.yaml.swapper;
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.parser.config.SQLParserRuleConfiguration;
 import org.apache.shardingsphere.parser.constant.SQLParserOrder;
 import org.apache.shardingsphere.parser.rule.builder.DefaultSQLParserRuleConfigurationBuilder;
@@ -35,7 +35,7 @@ import java.util.Optional;
  * TODO Rename YamlSQLTranslatorRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * YAML SQL parser rule configuration swapper.
  */
-public final class NewYamlSQLParserRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<SQLParserRuleConfiguration> {
+public final class NewYamlSQLParserRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<SQLParserRuleConfiguration> {
     
     private final YamlSQLParserCacheOptionConfigurationSwapper cacheOptionSwapper = new YamlSQLParserCacheOptionConfigurationSwapper();
     
diff --git a/kernel/sql-parser/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/sql-parser/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/sql-parser/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/sql-parser/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/sql-translator/core/src/main/java/org/apache/shardingsphere/sqltranslator/yaml/swapper/NewYamlSQLTranslatorRuleConfigurationSwapper.java b/kernel/sql-translator/core/src/main/java/org/apache/shardingsphere/sqltranslator/yaml/swapper/NewYamlSQLTranslatorRuleConfigurationSwapper.java
index e6c375c10bd..76a448b3081 100644
--- a/kernel/sql-translator/core/src/main/java/org/apache/shardingsphere/sqltranslator/yaml/swapper/NewYamlSQLTranslatorRuleConfigurationSwapper.java
+++ b/kernel/sql-translator/core/src/main/java/org/apache/shardingsphere/sqltranslator/yaml/swapper/NewYamlSQLTranslatorRuleConfigurationSwapper.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.sqltranslator.yaml.swapper;
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.sqltranslator.api.config.SQLTranslatorRuleConfiguration;
 import org.apache.shardingsphere.sqltranslator.constant.SQLTranslatorOrder;
 import org.apache.shardingsphere.sqltranslator.yaml.config.YamlSQLTranslatorRuleConfiguration;
@@ -33,7 +33,7 @@ import java.util.Optional;
  * TODO Rename YamlSQLTranslatorRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * YAML SQL translator rule configuration swapper.
  */
-public final class NewYamlSQLTranslatorRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<SQLTranslatorRuleConfiguration> {
+public final class NewYamlSQLTranslatorRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<SQLTranslatorRuleConfiguration> {
     
     @Override
     public Collection<YamlDataNode> swapToDataNodes(final SQLTranslatorRuleConfiguration data) {
diff --git a/kernel/sql-translator/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/sql-translator/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/sql-translator/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/sql-translator/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/traffic/core/src/main/java/org/apache/shardingsphere/traffic/yaml/swapper/NewYamlTrafficRuleConfigurationSwapper.java b/kernel/traffic/core/src/main/java/org/apache/shardingsphere/traffic/yaml/swapper/NewYamlTrafficRuleConfigurationSwapper.java
index ada13638c32..6908d7ce2f4 100644
--- a/kernel/traffic/core/src/main/java/org/apache/shardingsphere/traffic/yaml/swapper/NewYamlTrafficRuleConfigurationSwapper.java
+++ b/kernel/traffic/core/src/main/java/org/apache/shardingsphere/traffic/yaml/swapper/NewYamlTrafficRuleConfigurationSwapper.java
@@ -21,7 +21,7 @@ import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
 import org.apache.shardingsphere.infra.yaml.config.swapper.algorithm.YamlAlgorithmConfigurationSwapper;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.traffic.api.config.TrafficRuleConfiguration;
 import org.apache.shardingsphere.traffic.constant.TrafficOrder;
 import org.apache.shardingsphere.traffic.yaml.config.YamlTrafficRuleConfiguration;
@@ -36,7 +36,7 @@ import java.util.Optional;
  * TODO Rename YamlTrafficRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * New YAML traffic rule configuration swapper.
  */
-public final class NewYamlTrafficRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<TrafficRuleConfiguration> {
+public final class NewYamlTrafficRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<TrafficRuleConfiguration> {
     
     private final YamlTrafficStrategyConfigurationSwapper strategySwapper = new YamlTrafficStrategyConfigurationSwapper();
     
diff --git a/kernel/traffic/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/traffic/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/traffic/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/traffic/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
diff --git a/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/yaml/swapper/NewYamlTransactionRuleConfigurationSwapper.java b/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/yaml/swapper/NewYamlTransactionRuleConfigurationSwapper.java
index 7faacd87633..89d72f2f9b1 100644
--- a/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/yaml/swapper/NewYamlTransactionRuleConfigurationSwapper.java
+++ b/kernel/transaction/core/src/main/java/org/apache/shardingsphere/transaction/yaml/swapper/NewYamlTransactionRuleConfigurationSwapper.java
@@ -20,7 +20,7 @@ package org.apache.shardingsphere.transaction.yaml.swapper;
 import org.apache.shardingsphere.infra.config.converter.GlobalRuleNodeConverter;
 import org.apache.shardingsphere.infra.util.yaml.YamlEngine;
 import org.apache.shardingsphere.infra.util.yaml.datanode.YamlDataNode;
-import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper;
+import org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper;
 import org.apache.shardingsphere.transaction.config.TransactionRuleConfiguration;
 import org.apache.shardingsphere.transaction.constant.TransactionOrder;
 import org.apache.shardingsphere.transaction.yaml.config.YamlTransactionRuleConfiguration;
@@ -34,7 +34,7 @@ import java.util.Properties;
  * TODO Rename YamlTransactionRuleConfigurationSwapper when metadata structure adjustment completed. #25485
  * New YAML Transaction rule configuration swapper.
  */
-public final class NewYamlTransactionRuleConfigurationSwapper implements NewYamlRuleConfigurationSwapper<TransactionRuleConfiguration> {
+public final class NewYamlTransactionRuleConfigurationSwapper implements NewYamGlobalRuleConfigurationSwapper<TransactionRuleConfiguration> {
     
     @Override
     public Collection<YamlDataNode> swapToDataNodes(final TransactionRuleConfiguration data) {
diff --git a/kernel/transaction/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper b/kernel/transaction/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper
similarity index 100%
rename from kernel/transaction/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamlRuleConfigurationSwapper
rename to kernel/transaction/core/src/main/resources/META-INF/services/org.apache.shardingsphere.infra.yaml.config.swapper.rule.NewYamGlobalRuleConfigurationSwapper