You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@dubbo.apache.org by li...@apache.org on 2021/06/18 09:15:50 UTC

[dubbo] branch 3.0 updated: Refactor NameMapping to Metadata, Support application field in MigrationRule (#8089)

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

liujun pushed a commit to branch 3.0
in repository https://gitbox.apache.org/repos/asf/dubbo.git


The following commit(s) were added to refs/heads/3.0 by this push:
     new 2bca144  Refactor NameMapping to Metadata, Support application field in MigrationRule (#8089)
2bca144 is described below

commit 2bca144bfb4312f3e414c5551c233c4f93645fe6
Author: Albumen Kevin <jh...@gmail.com>
AuthorDate: Fri Jun 18 17:15:27 2021 +0800

    Refactor NameMapping to Metadata, Support application field in MigrationRule (#8089)
---
 .../dubbo/common/url/component/PathURLAddress.java |   3 +
 .../dubbo/metadata/AbstractServiceNameMapping.java | 110 +++++++++++++++
 .../apache/dubbo/metadata/ServiceNameMapping.java  |  20 ++-
 .../dubbo/metadata/report/MetadataReport.java      |   4 +
 .../metadata/store/nacos/NacosMetadataReport.java  |   8 +-
 .../store/zookeeper/ZookeeperMetadataReport.java   |   6 +
 .../registry/client/ServiceDiscoveryRegistry.java  |  60 +-------
 .../metadata/MetadataServiceNameMapping.java       |  17 ++-
 .../DefaultMigrationAddressComparator.java         |   3 +-
 .../client/migration/MigrationInvoker.java         |   8 +-
 .../client/migration/MigrationRuleHandler.java     |  13 +-
 .../client/migration/model/MigrationRule.java      | 155 +++++++++++++++------
 ...aceMigrationRule.java => SubMigrationRule.java} |  10 +-
 .../DefaultMigrationAddressComparatorTest.java     |   2 +-
 .../client/migration/MigrationRuleHandlerTest.java |   6 +-
 .../migration/MigrationRuleListenerTest.java       |   2 +-
 .../client/migration/model/MigrationRuleTest.java  |  82 +++++++----
 17 files changed, 346 insertions(+), 163 deletions(-)

diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/url/component/PathURLAddress.java b/dubbo-common/src/main/java/org/apache/dubbo/common/url/component/PathURLAddress.java
index f723991..ae0ea88 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/url/component/PathURLAddress.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/url/component/PathURLAddress.java
@@ -45,6 +45,9 @@ public class PathURLAddress extends URLAddress {
         while (path != null && path.startsWith("/")) {
             path = path.substring(1);
         }
+        if (path != null) {
+            path = path.intern();
+        }
         this.path = path;
     }
 
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/AbstractServiceNameMapping.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/AbstractServiceNameMapping.java
new file mode 100644
index 0000000..9596821
--- /dev/null
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/AbstractServiceNameMapping.java
@@ -0,0 +1,110 @@
+/*
+ * 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.dubbo.metadata;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.common.utils.StringUtils;
+
+import java.util.Set;
+import java.util.TreeSet;
+
+import static java.util.Collections.emptySet;
+import static java.util.Collections.unmodifiableSet;
+import static java.util.stream.Collectors.toSet;
+import static java.util.stream.Stream.of;
+import static org.apache.dubbo.common.constants.RegistryConstants.PROVIDED_BY;
+import static org.apache.dubbo.common.constants.RegistryConstants.SUBSCRIBED_SERVICE_NAMES_KEY;
+import static org.apache.dubbo.common.utils.CollectionUtils.isEmpty;
+import static org.apache.dubbo.common.utils.StringUtils.isBlank;
+
+public abstract class AbstractServiceNameMapping implements ServiceNameMapping {
+    protected final Logger logger = LoggerFactory.getLogger(getClass());
+
+    /**
+     * Get the service names from the specified Dubbo service interface, group, version and protocol
+     *
+     * @return
+     */
+    abstract public Set<String> get(URL url);
+
+    /**
+     * Get the service names from the specified Dubbo service interface, group, version and protocol
+     *
+     * @return
+     */
+    abstract public Set<String> getAndListen(URL url, MappingListener mappingListener);
+
+    @Override
+    public Set<String> getServices(URL subscribedURL) {
+        Set<String> subscribedServices = new TreeSet<>();
+
+        String serviceNames = subscribedURL.getParameter(PROVIDED_BY);
+        if (StringUtils.isNotEmpty(serviceNames)) {
+            logger.info(subscribedURL.getServiceInterface() + " mapping to " + serviceNames + " instructed by provided-by set by user.");
+            subscribedServices.addAll(parseServices(serviceNames));
+        }
+
+        if (isEmpty(subscribedServices)) {
+            subscribedServices = WritableMetadataService.getDefaultExtension().getCachedMapping(ServiceNameMapping.buildMappingKey(subscribedURL));
+        }
+
+        if (isEmpty(subscribedServices)) {
+            Set<String> mappedServices = get(subscribedURL);
+            logger.info(subscribedURL.getServiceInterface() + " mapping to " + mappedServices + " instructed by remote metadata center.");
+            subscribedServices.addAll(mappedServices);
+            WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(subscribedURL), subscribedServices);
+        }
+        return subscribedServices;
+    }
+
+    @Override
+    public Set<String> getAndListenServices(URL registryURL, URL subscribedURL, MappingListener listener) {
+        Set<String> subscribedServices = new TreeSet<>();
+        Set<String> globalConfiguredSubscribingServices = parseServices(registryURL.getParameter(SUBSCRIBED_SERVICE_NAMES_KEY));
+
+        String serviceNames = subscribedURL.getParameter(PROVIDED_BY);
+        if (StringUtils.isNotEmpty(serviceNames)) {
+            logger.info(subscribedURL.getServiceInterface() + " mapping to " + serviceNames + " instructed by provided-by set by user.");
+            subscribedServices.addAll(parseServices(serviceNames));
+        }
+
+        if (isEmpty(subscribedServices)) {
+            Set<String> mappedServices = getAndListen(subscribedURL, listener);
+            logger.info(subscribedURL.getServiceInterface() + " mapping to " + mappedServices + " instructed by remote metadata center.");
+            subscribedServices.addAll(mappedServices);
+            if (isEmpty(subscribedServices)) {
+                logger.info(subscribedURL.getServiceInterface() + " mapping to " + globalConfiguredSubscribingServices + " by default.");
+                subscribedServices.addAll(globalConfiguredSubscribingServices);
+            }
+        }
+
+        WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(subscribedURL), subscribedServices);
+
+        return subscribedServices;
+    }
+
+    static Set<String> parseServices(String literalServices) {
+        return isBlank(literalServices) ? emptySet() :
+            unmodifiableSet(of(literalServices.split(","))
+                .map(String::trim)
+                .filter(StringUtils::isNotEmpty)
+                .collect(toSet()));
+    }
+
+}
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/ServiceNameMapping.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/ServiceNameMapping.java
index 7a2d29b..13626be 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/ServiceNameMapping.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/ServiceNameMapping.java
@@ -45,13 +45,6 @@ public interface ServiceNameMapping {
     void map(URL url);
 
     /**
-     * Get the service names from the specified Dubbo service interface, group, version and protocol
-     *
-     * @return
-     */
-    Set<String> getAndListen(URL url, MappingListener mappingListener);
-
-    /**
      * Get the default extension of {@link ServiceNameMapping}
      *
      * @return non-null {@link ServiceNameMapping}
@@ -79,4 +72,17 @@ public interface ServiceNameMapping {
         }
         return new HashSet<>(Arrays.asList(content.split(COMMA_SEPARATOR)));
     }
+
+    /**
+     * 1.developer explicitly specifies the application name this interface belongs to
+     * 2.check Interface-App mapping
+     */
+    Set<String> getServices(URL subscribedURL);
+
+    /**
+     * 1.developer explicitly specifies the application name this interface belongs to
+     * 2.check Interface-App mapping
+     * 3.use the services specified in registry url.
+     */
+    Set<String> getAndListenServices(URL registryURL, URL subscribedURL, MappingListener listener);
 }
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/report/MetadataReport.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/report/MetadataReport.java
index ea3e7c8..38f1ded 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/report/MetadataReport.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/report/MetadataReport.java
@@ -80,4 +80,8 @@ public interface MetadataReport {
     default Set<String> getServiceAppMapping(String serviceKey, MappingListener listener, URL url) {
         return Collections.emptySet();
     }
+
+    default Set<String> getServiceAppMapping(String serviceKey, URL url) {
+        return Collections.emptySet();
+    }
 }
diff --git a/dubbo-metadata/dubbo-metadata-report-nacos/src/main/java/org/apache/dubbo/metadata/store/nacos/NacosMetadataReport.java b/dubbo-metadata/dubbo-metadata-report-nacos/src/main/java/org/apache/dubbo/metadata/store/nacos/NacosMetadataReport.java
index 401b159..cba0e52 100644
--- a/dubbo-metadata/dubbo-metadata-report-nacos/src/main/java/org/apache/dubbo/metadata/store/nacos/NacosMetadataReport.java
+++ b/dubbo-metadata/dubbo-metadata-report-nacos/src/main/java/org/apache/dubbo/metadata/store/nacos/NacosMetadataReport.java
@@ -26,8 +26,8 @@ import org.apache.dubbo.common.utils.MD5Utils;
 import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.metadata.MappingChangedEvent;
 import org.apache.dubbo.metadata.MappingListener;
-import org.apache.dubbo.metadata.ServiceNameMapping;
 import org.apache.dubbo.metadata.MetadataInfo;
+import org.apache.dubbo.metadata.ServiceNameMapping;
 import org.apache.dubbo.metadata.report.identifier.BaseMetadataIdentifier;
 import org.apache.dubbo.metadata.report.identifier.KeyTypeEnum;
 import org.apache.dubbo.metadata.report.identifier.MetadataIdentifier;
@@ -242,6 +242,12 @@ public class NacosMetadataReport extends AbstractMetadataReport {
         return ServiceNameMapping.getAppNames(content);
     }
 
+    @Override
+    public Set<String> getServiceAppMapping(String serviceKey, URL url) {
+        String content = getConfig(serviceKey, DEFAULT_MAPPING_GROUP);
+        return ServiceNameMapping.getAppNames(content);
+    }
+
     private String getConfig(String dataId, String group) {
         try {
             return configService.getConfig(dataId, group);
diff --git a/dubbo-metadata/dubbo-metadata-report-zookeeper/src/main/java/org/apache/dubbo/metadata/store/zookeeper/ZookeeperMetadataReport.java b/dubbo-metadata/dubbo-metadata-report-zookeeper/src/main/java/org/apache/dubbo/metadata/store/zookeeper/ZookeeperMetadataReport.java
index cbc2224..1e36894 100644
--- a/dubbo-metadata/dubbo-metadata-report-zookeeper/src/main/java/org/apache/dubbo/metadata/store/zookeeper/ZookeeperMetadataReport.java
+++ b/dubbo-metadata/dubbo-metadata-report-zookeeper/src/main/java/org/apache/dubbo/metadata/store/zookeeper/ZookeeperMetadataReport.java
@@ -162,6 +162,12 @@ public class ZookeeperMetadataReport extends AbstractMetadataReport {
     }
 
     @Override
+    public Set<String> getServiceAppMapping(String serviceKey, URL url) {
+        String path = buildPathKey(DEFAULT_MAPPING_GROUP, serviceKey);
+        return getAppNames(zkClient.getContent(path));
+    }
+
+    @Override
     public ConfigItem getConfigItem(String serviceKey, String group) {
         String path = buildPathKey(group, serviceKey);
         return zkClient.getConfigItem(path);
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/ServiceDiscoveryRegistry.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/ServiceDiscoveryRegistry.java
index 36f1990..4ca66ad 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/ServiceDiscoveryRegistry.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/ServiceDiscoveryRegistry.java
@@ -22,7 +22,6 @@ import org.apache.dubbo.common.extension.SPI;
 import org.apache.dubbo.common.logger.Logger;
 import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.common.utils.CollectionUtils;
-import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.metadata.MappingChangedEvent;
 import org.apache.dubbo.metadata.MappingListener;
 import org.apache.dubbo.metadata.ServiceNameMapping;
@@ -42,15 +41,10 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import java.util.TreeSet;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.stream.Collectors;
 
 import static java.lang.String.format;
-import static java.util.Collections.emptySet;
-import static java.util.Collections.unmodifiableSet;
-import static java.util.stream.Collectors.toSet;
-import static java.util.stream.Stream.of;
 import static org.apache.dubbo.common.constants.CommonConstants.CHECK_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.DUBBO;
 import static org.apache.dubbo.common.constants.CommonConstants.GROUP_CHAR_SEPARATOR;
@@ -59,14 +53,10 @@ import static org.apache.dubbo.common.constants.CommonConstants.INTERFACE_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.PROTOCOL_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.PROVIDER_SIDE;
 import static org.apache.dubbo.common.constants.CommonConstants.VERSION_KEY;
-import static org.apache.dubbo.common.constants.RegistryConstants.PROVIDED_BY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_CLUSTER_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_TYPE_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.SERVICE_REGISTRY_TYPE;
-import static org.apache.dubbo.common.constants.RegistryConstants.SUBSCRIBED_SERVICE_NAMES_KEY;
 import static org.apache.dubbo.common.function.ThrowableAction.execute;
-import static org.apache.dubbo.common.utils.CollectionUtils.isEmpty;
-import static org.apache.dubbo.common.utils.StringUtils.isBlank;
 import static org.apache.dubbo.metadata.ServiceNameMapping.toStringKeys;
 import static org.apache.dubbo.registry.client.ServiceDiscoveryFactory.getExtension;
 
@@ -227,8 +217,8 @@ public class ServiceDiscoveryRegistry implements Registry {
 
         Set<String> subscribedServices = Collections.emptySet();
         try {
-            subscribedServices = getServices(registryURL, url, listener);
-            WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(url), subscribedServices);
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            subscribedServices = serviceNameMapping.getAndListenServices(registryURL, url, new DefaultMappingListener(url, subscribedServices, listener));
         } catch (Exception e) {
             logger.warn("Cannot find app mapping for service " + url.getServiceInterface() + ", will not migrate.", e);
         }
@@ -432,52 +422,6 @@ public class ServiceDiscoveryRegistry implements Registry {
         return serviceListeners;
     }
 
-
-    /**
-     * 1.developer explicitly specifies the application name this interface belongs to
-     * 2.check Interface-App mapping
-     * 3.use the services specified in registry url.
-     *
-     * @param subscribedURL
-     * @return
-     */
-    protected Set<String> getServices(URL registryURL, URL subscribedURL, NotifyListener listener) {
-        Set<String> subscribedServices = new TreeSet<>();
-        Set<String> globalConfiguredSubscribingServices = parseServices(registryURL.getParameter(SUBSCRIBED_SERVICE_NAMES_KEY));
-
-        String serviceNames = subscribedURL.getParameter(PROVIDED_BY);
-        if (StringUtils.isNotEmpty(serviceNames)) {
-            logger.info(subscribedURL.getServiceInterface() + " mapping to " + serviceNames + " instructed by provided-by set by user.");
-            subscribedServices.addAll(parseServices(serviceNames));
-        }
-
-        if (isEmpty(subscribedServices)) {
-            Set<String> mappedServices = findMappedServices(registryURL, subscribedURL, new DefaultMappingListener(subscribedURL, subscribedServices, listener));
-            logger.info(subscribedURL.getServiceInterface() + " mapping to " + mappedServices + " instructed by remote metadata center.");
-            subscribedServices.addAll(mappedServices);
-            if (isEmpty(subscribedServices)) {
-                logger.info(subscribedURL.getServiceInterface() + " mapping to " + globalConfiguredSubscribingServices + " by default.");
-                subscribedServices.addAll(globalConfiguredSubscribingServices);
-            }
-        }
-        return subscribedServices;
-    }
-
-    protected Set<String> findMappedServices(URL registryURL, URL subscribedURL, MappingListener listener) {
-        Set<String> result = new LinkedHashSet<>();
-        ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
-        result.addAll(serviceNameMapping.getAndListen(subscribedURL, listener));
-        return result;
-    }
-
-    public static Set<String> parseServices(String literalServices) {
-        return isBlank(literalServices) ? emptySet() :
-                unmodifiableSet(of(literalServices.split(","))
-                        .map(String::trim)
-                        .filter(StringUtils::isNotEmpty)
-                        .collect(toSet()));
-    }
-
     private class DefaultMappingListener implements MappingListener {
         private final Logger logger = LoggerFactory.getLogger(DefaultMappingListener.class);
         private URL url;
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceNameMapping.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceNameMapping.java
index 4db6fa6..4ad262c 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceNameMapping.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceNameMapping.java
@@ -21,8 +21,8 @@ import org.apache.dubbo.common.config.configcenter.ConfigItem;
 import org.apache.dubbo.common.logger.Logger;
 import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.common.utils.StringUtils;
+import org.apache.dubbo.metadata.AbstractServiceNameMapping;
 import org.apache.dubbo.metadata.MappingListener;
-import org.apache.dubbo.metadata.ServiceNameMapping;
 import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.report.MetadataReport;
 import org.apache.dubbo.metadata.report.MetadataReportInstance;
@@ -37,7 +37,7 @@ import static org.apache.dubbo.common.constants.CommonConstants.COMMA_SEPARATOR;
 import static org.apache.dubbo.common.constants.CommonConstants.DEFAULT_KEY;
 import static org.apache.dubbo.rpc.model.ApplicationModel.getName;
 
-public class MetadataServiceNameMapping implements ServiceNameMapping {
+public class MetadataServiceNameMapping extends AbstractServiceNameMapping {
 
     private final Logger logger = LoggerFactory.getLogger(getClass());
 
@@ -74,6 +74,19 @@ public class MetadataServiceNameMapping implements ServiceNameMapping {
     }
 
     @Override
+    public Set<String> get(URL url) {
+        Set<String> serviceNames = new LinkedHashSet<>();
+        execute(() -> {
+            String serviceInterface = url.getServiceInterface();
+            String registryCluster = getRegistryCluster(url);
+            MetadataReport metadataReport = MetadataReportInstance.getMetadataReport(registryCluster);
+            Set<String> apps = metadataReport.getServiceAppMapping(serviceInterface, url);
+            serviceNames.addAll(apps);
+        });
+        return serviceNames;
+    }
+
+    @Override
     public Set<String> getAndListen(URL url, MappingListener mappingListener) {
         Set<String> serviceNames = new LinkedHashSet<>();
         execute(() -> {
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparator.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparator.java
index 5223933..299602e 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparator.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparator.java
@@ -64,8 +64,7 @@ public class DefaultMigrationAddressComparator implements MigrationAddressCompar
         migrationData.put(NEW_ADDRESS_SIZE, newAddressSize);
 
         String rawThreshold = null;
-        String serviceKey = oldInvoker.getUrl().getDisplayServiceKey();
-        Float configedThreshold = rule == null ? null : rule.getThreshold(serviceKey);
+        Float configedThreshold = rule == null ? null : rule.getThreshold(oldInvoker.getUrl());
         if (configedThreshold != null && configedThreshold >= 0) {
             rawThreshold = String.valueOf(configedThreshold);
         }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationInvoker.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationInvoker.java
index c6c33f3..0c6aff4 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationInvoker.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationInvoker.java
@@ -174,7 +174,7 @@ public class MigrationInvoker<T> implements MigrationClusterInvoker<T> {
         // wait and compare threshold
         waitAddressNotify(newRule, latch);
 
-        if (Boolean.TRUE.equals(newRule.getForce(consumerUrl.getDisplayServiceKey()))) {
+        if (Boolean.TRUE.equals(newRule.getForce(consumerUrl))) {
             // force migrate, ignore threshold check
             this.currentAvailableInvoker = invoker;
             this.destroyServiceDiscoveryInvoker();
@@ -211,7 +211,7 @@ public class MigrationInvoker<T> implements MigrationClusterInvoker<T> {
         // wait and compare threshold
         waitAddressNotify(newRule, latch);
 
-        if (Boolean.TRUE.equals(newRule.getForce(consumerUrl.getDisplayServiceKey()))) {
+        if (Boolean.TRUE.equals(newRule.getForce(consumerUrl))) {
             // force migrate, ignore threshold check
             this.currentAvailableInvoker = serviceDiscoveryInvoker;
             this.destroyInterfaceInvoker();
@@ -247,7 +247,7 @@ public class MigrationInvoker<T> implements MigrationClusterInvoker<T> {
 
     private void waitAddressNotify(MigrationRule newRule, CountDownLatch latch) {
         // wait and compare threshold
-        Integer delay = newRule.getDelay(consumerUrl.getDisplayServiceKey());
+        Integer delay = newRule.getDelay(consumerUrl);
         if (delay != null) {
             try {
                 Thread.sleep(delay * 1000);
@@ -397,7 +397,7 @@ public class MigrationInvoker<T> implements MigrationClusterInvoker<T> {
     @Override
     public void setMigrationRule(MigrationRule rule) {
         this.rule = rule;
-        promotion = Optional.ofNullable(rule.getProportion(consumerUrl.getDisplayServiceKey())).orElse(100);
+        promotion = Optional.ofNullable(rule.getProportion(consumerUrl)).orElse(100);
     }
 
     protected void destroyServiceDiscoveryInvoker() {
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandler.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandler.java
index 2bf90d6..410776c 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandler.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandler.java
@@ -54,9 +54,8 @@ public class MigrationRuleHandler<T> {
                             ConfigurationUtils.getCachedDynamicProperty(DUBBO_SERVICEDISCOVERY_MIGRATION, step.name())));
         } else {
             try {
-                String serviceKey = consumerURL.getDisplayServiceKey();
-                step = getMigrationStep(rule, step, serviceKey);
-                threshold = getMigrationThreshold(rule, threshold, serviceKey);
+                step = getMigrationStep(rule, step);
+                threshold = getMigrationThreshold(rule, threshold);
             } catch (Exception e) {
                 logger.error("Failed to get step and threshold info from rule: " + rule, e);
             }
@@ -117,14 +116,14 @@ public class MigrationRuleHandler<T> {
         this.migrationInvoker.setMigrationRule(rule);
     }
 
-    private MigrationStep getMigrationStep(MigrationRule rule, MigrationStep step, String serviceKey) {
-        MigrationStep configuredStep = rule.getStep(serviceKey);
+    private MigrationStep getMigrationStep(MigrationRule rule, MigrationStep step) {
+        MigrationStep configuredStep = rule.getStep(consumerURL);
         step = configuredStep == null ? step : configuredStep;
         return step;
     }
 
-    private Float getMigrationThreshold(MigrationRule rule, Float threshold, String serviceKey) {
-        Float configuredThreshold = rule.getThreshold(serviceKey);
+    private Float getMigrationThreshold(MigrationRule rule, Float threshold) {
+        Float configuredThreshold = rule.getThreshold(consumerURL);
         threshold = configuredThreshold == null ? threshold : configuredThreshold;
         return threshold;
     }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/MigrationRule.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/MigrationRule.java
index 199d702..48b7528 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/MigrationRule.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/MigrationRule.java
@@ -16,16 +16,18 @@
  */
 package org.apache.dubbo.registry.client.migration.model;
 
+import org.apache.dubbo.common.URL;
 import org.apache.dubbo.common.utils.CollectionUtils;
+import org.apache.dubbo.metadata.ServiceNameMapping;
 
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.Constructor;
 import org.yaml.snakeyaml.constructor.SafeConstructor;
 
-import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 import java.util.stream.Collectors;
 
 /**
@@ -47,6 +49,13 @@ import java.util.stream.Collectors;
  *     step: APPLICATION_FIRST
  *   - serviceKey: GreetingService:1.0.0
  *     step: FORCE_APPLICATION
+ * applications:
+ *  - serviceKey: TestApplication
+ *    threshold: 0.3
+ *    proportion: 20
+ *    delay: 10
+ *    force: false
+ *    step: FORCE_INTERFACE
  */
 public class MigrationRule {
     public static final MigrationRule INIT = new MigrationRule();
@@ -57,9 +66,11 @@ public class MigrationRule {
     private Integer proportion;
     private Integer delay;
     private Boolean force;
-    private List<InterfaceMigrationRule> interfaces;
+    private List<SubMigrationRule> interfaces;
+    private List<SubMigrationRule> applications;
 
-    private transient Map<String, InterfaceMigrationRule> interfaceRules;
+    private transient Map<String, SubMigrationRule> interfaceRules;
+    private transient Map<String, SubMigrationRule> applicationRules;
 
     @SuppressWarnings("unchecked")
     private static MigrationRule parseFromMap(Map<String, Object> map) {
@@ -94,7 +105,13 @@ public class MigrationRule {
         Object interfaces = map.get("interfaces");
         if (interfaces != null && List.class.isAssignableFrom(interfaces.getClass())) {
             migrationRule.setInterfaces(((List<Map<String, Object>>) interfaces).stream()
-                    .map(InterfaceMigrationRule::parseFromMap).collect(Collectors.toList()));
+                    .map(SubMigrationRule::parseFromMap).collect(Collectors.toList()));
+        }
+
+        Object applications = map.get("applications");
+        if (applications != null && List.class.isAssignableFrom(applications.getClass())) {
+            migrationRule.setApplications(((List<Map<String, Object>>) applications).stream()
+                .map(SubMigrationRule::parseFromMap).collect(Collectors.toList()));
         }
 
         return migrationRule;
@@ -115,35 +132,55 @@ public class MigrationRule {
         this.key = key;
     }
 
-    public MigrationStep getStep(String serviceKey) {
+    public MigrationStep getStep(URL consumerURL) {
         if (interfaceRules != null) {
-            InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
+            SubMigrationRule rule = interfaceRules.get(consumerURL.getDisplayServiceKey());
             if (rule != null) {
                 return rule.getStep() == null ? step : rule.getStep();
             }
         }
 
-        return step;
-    }
-
-    public InterfaceMigrationRule getInterfaceRule(String serviceKey) {
-        if (interfaceRules == null) {
-            return null;
+        if (applications != null) {
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            Set<String> services = serviceNameMapping.getServices(consumerURL);
+            if(CollectionUtils.isNotEmpty(services)) {
+                for (String service : services) {
+                    SubMigrationRule rule = applicationRules.get(service);
+                    if (rule != null) {
+                        return rule.getStep() == null ? step : rule.getStep();
+                    }
+                }
+            }
         }
-        return interfaceRules.get(serviceKey);
+
+        return step;
     }
 
     public MigrationStep getStep() {
         return step;
     }
 
-    public Float getThreshold(String serviceKey) {
+    public Float getThreshold(URL consumerURL) {
         if (interfaceRules != null) {
-            InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
+            SubMigrationRule rule = interfaceRules.get(consumerURL.getDisplayServiceKey());
             if (rule != null) {
                 return rule.getThreshold() == null ? threshold : rule.getThreshold();
             }
         }
+
+        if (applications != null) {
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            Set<String> services = serviceNameMapping.getServices(consumerURL);
+            if(CollectionUtils.isNotEmpty(services)) {
+                for (String service : services) {
+                    SubMigrationRule rule = applicationRules.get(service);
+                    if (rule != null) {
+                        return rule.getThreshold() == null ? threshold : rule.getThreshold();
+                    }
+                }
+            }
+        }
+
         return threshold;
     }
 
@@ -159,13 +196,27 @@ public class MigrationRule {
         return proportion;
     }
 
-    public Integer getProportion(String serviceKey) {
+    public Integer getProportion(URL consumerURL) {
         if (interfaceRules != null) {
-            InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
+            SubMigrationRule rule = interfaceRules.get(consumerURL.getDisplayServiceKey());
             if (rule != null) {
                 return rule.getProportion() == null ? proportion : rule.getProportion();
             }
         }
+
+        if (applications != null) {
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            Set<String> services = serviceNameMapping.getServices(consumerURL);
+            if(CollectionUtils.isNotEmpty(services)) {
+                for (String service : services) {
+                    SubMigrationRule rule = applicationRules.get(service);
+                    if (rule != null) {
+                        return rule.getProportion() == null ? proportion : rule.getProportion();
+                    }
+                }
+            }
+        }
+
         return proportion;
     }
 
@@ -177,13 +228,27 @@ public class MigrationRule {
         return delay;
     }
 
-    public Integer getDelay(String serviceKey) {
+    public Integer getDelay(URL consumerURL) {
         if (interfaceRules != null) {
-            InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
+            SubMigrationRule rule = interfaceRules.get(consumerURL.getDisplayServiceKey());
             if (rule != null) {
                 return rule.getDelay() == null ? delay : rule.getDelay();
             }
         }
+
+        if (applications != null) {
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            Set<String> services = serviceNameMapping.getServices(consumerURL);
+            if(CollectionUtils.isNotEmpty(services)) {
+                for (String service : services) {
+                    SubMigrationRule rule = applicationRules.get(service);
+                    if (rule != null) {
+                        return rule.getDelay() == null ? delay : rule.getDelay();
+                    }
+                }
+            }
+        }
+
         return delay;
     }
 
@@ -199,13 +264,27 @@ public class MigrationRule {
         return force;
     }
 
-    public Boolean getForce(String serviceKey) {
+    public Boolean getForce(URL consumerURL) {
         if (interfaceRules != null) {
-            InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
+            SubMigrationRule rule = interfaceRules.get(consumerURL.getDisplayServiceKey());
             if (rule != null) {
                 return rule.getForce() == null ? force : rule.getForce();
             }
         }
+
+        if (applications != null) {
+            ServiceNameMapping serviceNameMapping = ServiceNameMapping.getDefaultExtension();
+            Set<String> services = serviceNameMapping.getServices(consumerURL);
+            if(CollectionUtils.isNotEmpty(services)) {
+                for (String service : services) {
+                    SubMigrationRule rule = applicationRules.get(service);
+                    if (rule != null) {
+                        return rule.getForce() == null ? force : rule.getForce();
+                    }
+                }
+            }
+        }
+
         return force;
     }
 
@@ -213,11 +292,11 @@ public class MigrationRule {
         this.force = force;
     }
 
-    public List<InterfaceMigrationRule> getInterfaces() {
+    public List<SubMigrationRule> getInterfaces() {
         return interfaces;
     }
 
-    public void setInterfaces(List<InterfaceMigrationRule> interfaces) {
+    public void setInterfaces(List<SubMigrationRule> interfaces) {
         this.interfaces = interfaces;
         if (interfaces != null) {
             this.interfaceRules = new HashMap<>();
@@ -227,31 +306,19 @@ public class MigrationRule {
         }
     }
 
-    public boolean removeInterfaceRule(String serviceKey) {
-        if (CollectionUtils.isNotEmpty(this.interfaces)) {
-            boolean removed = this.interfaces.removeIf(interfaceMigrationRule -> interfaceMigrationRule.getServiceKey().equals(serviceKey));
-            this.interfaceRules.remove(serviceKey);
-            return removed;
-        }
-        return false;
+    public List<SubMigrationRule> getApplications() {
+        return applications;
     }
 
-    public boolean addInterfaceRule(String serviceKey, MigrationStep step, Float threshold, Integer proportion) {
-        if (getInterfaceRule(serviceKey) != null) {
-            return false;
-        }
-
-        if (this.interfaces == null) {
-            this.interfaces = new ArrayList<>();
+    public void setApplications(List<SubMigrationRule> applications) {
+        this.applications = applications;
+        if (applications != null) {
+            this.applicationRules = new HashMap<>();
+            applications.forEach(rule -> {
+                applicationRules.put(rule.getServiceKey(), rule);
+            });
         }
-        InterfaceMigrationRule interfaceMigrationRule = new InterfaceMigrationRule(serviceKey, step, threshold, proportion);
-        this.interfaces.add(interfaceMigrationRule);
 
-        if (interfaceRules == null) {
-            this.interfaceRules = new HashMap<>();
-        }
-        this.interfaceRules.put(serviceKey, interfaceMigrationRule);
-        return true;
     }
 
     public static MigrationRule parse(String rawRule) {
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/InterfaceMigrationRule.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/SubMigrationRule.java
similarity index 89%
rename from dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/InterfaceMigrationRule.java
rename to dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/SubMigrationRule.java
index 79f112b..c64c9c9 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/InterfaceMigrationRule.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/SubMigrationRule.java
@@ -18,7 +18,7 @@ package org.apache.dubbo.registry.client.migration.model;
 
 import java.util.Map;
 
-public class InterfaceMigrationRule {
+public class SubMigrationRule {
     private String serviceKey;
     private MigrationStep step;
     private Float threshold;
@@ -26,8 +26,8 @@ public class InterfaceMigrationRule {
     private Integer delay;
     private Boolean force;
 
-    public static InterfaceMigrationRule parseFromMap(Map<String, Object> map) {
-        InterfaceMigrationRule interfaceMigrationRule = new InterfaceMigrationRule();
+    public static SubMigrationRule parseFromMap(Map<String, Object> map) {
+        SubMigrationRule interfaceMigrationRule = new SubMigrationRule();
         interfaceMigrationRule.setServiceKey((String) map.get("serviceKey"));
 
         Object step = map.get("step");
@@ -58,9 +58,9 @@ public class InterfaceMigrationRule {
         return interfaceMigrationRule;
     }
 
-    public InterfaceMigrationRule(){}
+    public SubMigrationRule(){}
 
-    public InterfaceMigrationRule(String serviceKey, MigrationStep step, Float threshold, Integer proportion) {
+    public SubMigrationRule(String serviceKey, MigrationStep step, Float threshold, Integer proportion) {
         this.serviceKey = serviceKey;
         this.step = step;
         this.threshold = threshold;
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparatorTest.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparatorTest.java
index 8ae2417..0314ff7 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparatorTest.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/DefaultMigrationAddressComparatorTest.java
@@ -82,7 +82,7 @@ public class DefaultMigrationAddressComparatorTest {
 
         Assertions.assertTrue(comparator.shouldMigrate(newInvoker, oldInvoker, null));
 
-        Mockito.when(rule.getThreshold("test")).thenReturn(0.5f);
+        Mockito.when(rule.getThreshold(url)).thenReturn(0.5f);
         newInvokerList.clear();
         newInvokerList.add(Mockito.mock(Invoker.class));
         Assertions.assertTrue(comparator.shouldMigrate(newInvoker, oldInvoker, rule));
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandlerTest.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandlerTest.java
index 83b72907..e4f4e06 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandlerTest.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleHandlerTest.java
@@ -39,15 +39,15 @@ public class MigrationRuleHandlerTest {
         Mockito.verify(invoker, Mockito.times(1)).migrateToApplicationFirstInvoker(MigrationRule.INIT);
 
         MigrationRule rule = Mockito.mock(MigrationRule.class);
-        Mockito.when(rule.getStep("test")).thenReturn(MigrationStep.FORCE_APPLICATION);
+        Mockito.when(rule.getStep(url)).thenReturn(MigrationStep.FORCE_APPLICATION);
         handler.doMigrate(rule);
         Mockito.verify(invoker, Mockito.times(1)).migrateToForceApplicationInvoker(rule);
 
-        Mockito.when(rule.getStep("test")).thenReturn(MigrationStep.APPLICATION_FIRST);
+        Mockito.when(rule.getStep(url)).thenReturn(MigrationStep.APPLICATION_FIRST);
         handler.doMigrate(rule);
         Mockito.verify(invoker, Mockito.times(1)).migrateToApplicationFirstInvoker(rule);
 
-        Mockito.when(rule.getStep("test")).thenReturn(MigrationStep.FORCE_INTERFACE);
+        Mockito.when(rule.getStep(url)).thenReturn(MigrationStep.FORCE_INTERFACE);
         handler.doMigrate(rule);
         Mockito.verify(invoker, Mockito.times(1)).migrateToForceInterfaceInvoker(rule);
     }
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleListenerTest.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleListenerTest.java
index f2f2bac..08582c3 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleListenerTest.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/MigrationRuleListenerTest.java
@@ -55,7 +55,7 @@ public class MigrationRuleListenerTest {
         Mockito.when(consumerURL.getServiceKey()).thenReturn("Test");
         Mockito.when(consumerURL.getParameter("timestamp")).thenReturn("1");
 
-        System.setProperty("dubbo.application.migration.delay", "100");
+        System.setProperty("dubbo.application.migration.delay", "1000");
         MigrationRuleHandler handler = Mockito.mock(MigrationRuleHandler.class);
 
         MigrationRuleListener migrationRuleListener = new MigrationRuleListener();
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/model/MigrationRuleTest.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/model/MigrationRuleTest.java
index 0b40e16..ad110e1 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/model/MigrationRuleTest.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/migration/model/MigrationRuleTest.java
@@ -16,53 +16,79 @@
  */
 package org.apache.dubbo.registry.client.migration.model;
 
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.metadata.ServiceNameMapping;
+import org.apache.dubbo.metadata.WritableMetadataService;
+
 import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import java.util.Collections;
 
 import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotNull;
 
 public class MigrationRuleTest {
 
     @Test
     public void test_parse() {
         String rule = "key: demo-consumer\n" +
-                "step: APPLICATION_FIRST\n" +
-                "threshold: 1.0\n" +
-                "proportion: 60\n" +
-                "delay: 60\n" +
-                "force: false\n" +
-                "interfaces:\n" +
-                "  - serviceKey: DemoService:1.0.0\n" +
-                "    threshold: 0.5\n" +
-                "    proportion: 30\n" +
-                "    delay: 30\n" +
-                "    force: true\n" +
-                "    step: APPLICATION_FIRST\n" +
-                "  - serviceKey: GreetingService:1.0.0\n" +
-                "    step: FORCE_APPLICATION";
+            "step: APPLICATION_FIRST\n" +
+            "threshold: 1.0\n" +
+            "proportion: 60\n" +
+            "delay: 60\n" +
+            "force: false\n" +
+            "interfaces:\n" +
+            "  - serviceKey: DemoService:1.0.0\n" +
+            "    threshold: 0.5\n" +
+            "    proportion: 30\n" +
+            "    delay: 30\n" +
+            "    force: true\n" +
+            "    step: APPLICATION_FIRST\n" +
+            "  - serviceKey: GreetingService:1.0.0\n" +
+            "    step: FORCE_APPLICATION\n" +
+            "applications:\n" +
+            "  - serviceKey: TestApplication\n" +
+            "    threshold: 0.3\n" +
+            "    proportion: 20\n" +
+            "    delay: 10\n" +
+            "    force: false\n" +
+            "    step: FORCE_INTERFACE\n";
 
         MigrationRule migrationRule = MigrationRule.parse(rule);
         assertEquals("demo-consumer", migrationRule.getKey());
-        assertEquals(MigrationStep.APPLICATION_FIRST ,migrationRule.getStep());
+        assertEquals(MigrationStep.APPLICATION_FIRST, migrationRule.getStep());
         assertEquals(1.0f, migrationRule.getThreshold());
         assertEquals(60, migrationRule.getProportion());
         assertEquals(60, migrationRule.getDelay());
         assertEquals(false, migrationRule.getForce());
 
+        URL url = Mockito.mock(URL.class);
+        Mockito.when(url.getDisplayServiceKey()).thenReturn("DemoService:1.0.0");
+
         assertEquals(migrationRule.getInterfaces().size(), 2);
-        assertNotNull(migrationRule.getInterfaceRule("DemoService:1.0.0"));
-        assertNotNull(migrationRule.getInterfaceRule("GreetingService:1.0.0"));
 
-        assertEquals(0.5f, migrationRule.getThreshold("DemoService:1.0.0"));
-        assertEquals(30, migrationRule.getProportion("DemoService:1.0.0"));
-        assertEquals(30, migrationRule.getDelay("DemoService:1.0.0"));
-        assertEquals(true, migrationRule.getForce("DemoService:1.0.0"));
-        assertEquals(MigrationStep.APPLICATION_FIRST ,migrationRule.getStep("DemoService:1.0.0"));
+        assertEquals(0.5f, migrationRule.getThreshold(url));
+        assertEquals(30, migrationRule.getProportion(url));
+        assertEquals(30, migrationRule.getDelay(url));
+        assertEquals(true, migrationRule.getForce(url));
+        assertEquals(MigrationStep.APPLICATION_FIRST, migrationRule.getStep(url));
+
+        Mockito.when(url.getDisplayServiceKey()).thenReturn("GreetingService:1.0.0");
+        assertEquals(1.0f, migrationRule.getThreshold(url));
+        assertEquals(60, migrationRule.getProportion(url));
+        assertEquals(60, migrationRule.getDelay(url));
+        assertEquals(false, migrationRule.getForce(url));
+        assertEquals(MigrationStep.FORCE_APPLICATION, migrationRule.getStep(url));
+
+        Mockito.when(url.getDisplayServiceKey()).thenReturn("GreetingService:1.0.1");
+        Mockito.when(url.getServiceInterface()).thenReturn("GreetingService");
+        WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(url), Collections.singleton("TestApplication"));
+        assertEquals(0.3f, migrationRule.getThreshold(url));
+        assertEquals(20, migrationRule.getProportion(url));
+        assertEquals(10, migrationRule.getDelay(url));
+        assertEquals(false, migrationRule.getForce(url));
+        assertEquals(MigrationStep.FORCE_INTERFACE, migrationRule.getStep(url));
 
-        assertEquals(1.0f, migrationRule.getThreshold("GreetingService:1.0.0"));
-        assertEquals(60, migrationRule.getProportion("GreetingService:1.0.0"));
-        assertEquals(60, migrationRule.getDelay("GreetingService:1.0.0"));
-        assertEquals(false, migrationRule.getForce("GreetingService:1.0.0"));
-        assertEquals(MigrationStep.FORCE_APPLICATION ,migrationRule.getStep("GreetingService:1.0.0"));
+        WritableMetadataService.getDefaultExtension().removeCachedMapping("GreetingService");
     }
 }