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/01/07 08:39:25 UTC

[dubbo] branch 3.0 updated: 3.0 migration rule enhancment(#7086)

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 c4815f5  3.0 migration rule  enhancment(#7086)
c4815f5 is described below

commit c4815f56025411512aa49d3e06388bc8f1cf23ba
Author: ken.lj <ke...@gmail.com>
AuthorDate: Thu Jan 7 16:39:06 2021 +0800

    3.0 migration rule  enhancment(#7086)
---
 .../cluster/filter/DefaultFilterChainBuilder.java  |  62 +++
 .../rpc/cluster/filter/FilterChainBuilder.java     | 148 +++++++
 .../filter/InvocationInterceptorBuilder.java       |   7 +-
 .../rpc/cluster/filter}/ProtocolFilterWrapper.java | 165 ++++----
 .../cluster/support/AbstractClusterInvoker.java    |  16 +-
 .../cluster/support/wrapper/AbstractCluster.java   | 107 ++++-
 .../dubbo/internal/org.apache.dubbo.rpc.Protocol   |   1 +
 ...che.dubbo.rpc.cluster.filter.FilterChainBuilder |   1 +
 .../src/main/java/org/apache/dubbo/common/URL.java |  14 +
 .../dubbo/common/constants/CommonConstants.java    |   4 +-
 .../dubbo/common/constants/RegistryConstants.java  |   6 +-
 .../apache/dubbo/common/extension/Activate.java    |   2 +
 .../dubbo/common/extension/ExtensionLoader.java    |  15 +
 .../manager/DefaultExecutorRepository.java         |   2 +-
 .../org/apache/dubbo/config/AbstractConfig.java    |   6 +
 .../dubbo/config/AbstractInterfaceConfig.java      |   8 +-
 .../org/apache/dubbo/config/ApplicationConfig.java |  16 +
 .../org/apache/dubbo/config/RegistryConfig.java    |  24 ++
 .../org/apache/dubbo/config/ServiceConfig.java     |   7 +-
 .../dubbo/config/bootstrap/DubboBootstrap.java     |  95 ++---
 .../ConfigurableMetadataServiceExporter.java       |   7 +-
 ...java => ServiceInstanceHostPortCustomizer.java} |  49 ++-
 .../dubbo/config/utils/ConfigValidationUtils.java  |  36 +-
 ...dubbo.registry.client.ServiceInstanceCustomizer |   2 +-
 .../src/main/resources/META-INF/compat/dubbo.xsd   |  11 +-
 .../src/main/resources/META-INF/dubbo.xsd          |  20 +
 dubbo-demo/dubbo-demo-generic-call/pom.xml         |  71 ++++
 .../dubbo/demo/consumer/GenericApplication.java    |  85 ++++
 .../src/main/resources/log4j.properties            |   7 +
 dubbo-demo/dubbo-demo-interface/pom.xml            |   7 +
 .../org/apache/dubbo/demo/RestDemoService.java     |  31 +-
 .../apache/dubbo/demo/consumer/Application.java    |   4 +-
 .../src/main/resources/spring/dubbo-consumer.xml   |   2 +-
 .../dubbo-demo-xml/dubbo-demo-xml-provider/pom.xml |   4 +
 .../dubbo/demo/provider/RestDemoServiceImpl.java   |  58 +++
 .../src/main/resources/spring/dubbo-provider.xml   |  10 +-
 dubbo-demo/pom.xml                                 |   1 +
 .../apache/dubbo/metadata/MetadataConstants.java   |   2 +-
 .../org/apache/dubbo/metadata/MetadataInfo.java    |  23 +-
 .../org/apache/dubbo/metadata/MetadataService.java |   5 +
 .../apache/dubbo/metadata/RevisionResolver.java    |   4 +
 .../apache/dubbo/metadata/ServiceNameMapping.java  |  14 +
 .../dubbo/metadata/WritableMetadataService.java    |  15 +
 .../store/zookeeper/ZookeeperMetadataReport.java   |  40 +-
 .../apache/dubbo/qos/command/impl/BaseOffline.java |  11 +-
 .../apache/dubbo/qos/command/impl/BaseOnline.java  |   9 +-
 .../org/apache/dubbo/qos/command/impl/Offline.java |  15 +-
 .../apache/dubbo/qos/command/impl/OfflineApp.java  |   9 +-
 .../dubbo/qos/command/impl/OfflineInterface.java   |   5 +-
 .../org/apache/dubbo/qos/command/impl/Online.java  |  13 +-
 .../apache/dubbo/qos/command/impl/OnlineApp.java   |  11 +-
 .../dubbo/qos/command/impl/OnlineInterface.java    |   5 +-
 .../dubbo/qos/command/impl/PublishMetadata.java    |   4 +-
 .../registry/client/DefaultServiceInstance.java    |  62 +++
 .../registry/client/ServiceDiscoveryRegistry.java  | 158 +------
 .../listener/ServiceInstancesChangedListener.java  |  30 +-
 ...MetadataServiceURLParamsMetadataCustomizer.java |  14 +-
 .../registry/client/metadata/MetadataUtils.java    |  14 +-
 .../metadata/ProtocolPortsMetadataCustomizer.java  |  11 +-
 .../ServiceInstanceMetadataCustomizer.java         |  16 +-
 .../metadata/ServiceInstanceMetadataUtils.java     | 107 ++---
 .../StandardMetadataServiceURLBuilder.java         |  41 +-
 .../store/InMemoryWritableMetadataService.java     |  64 ++-
 .../DefaultMigrationAddressComparator.java         |   6 +-
 .../client/migration/MigrationClusterInvoker.java  |   2 +
 .../client/migration/MigrationInvoker.java         |  10 +
 .../client/migration/MigrationRuleHandler.java     | 100 +++--
 .../client/migration/MigrationRuleListener.java    | 144 ++++++-
 ...tionRule.java => ApplicationMigrationRule.java} |  21 +-
 .../migration/model/InterfaceMigrationRule.java    |  18 +
 .../client/migration/model/MigrationRule.java      | 122 +++++-
 .../registry/integration/DynamicDirectory.java     |   4 +
 .../registry/integration/RegistryProtocol.java     |  16 +-
 .../integration/RegistryProtocolListener.java      |   2 +-
 .../dubbo/registry/support/AbstractRegistry.java   |   2 +-
 .../java/org/apache/dubbo/registry/ZKTools.java    |  33 +-
 .../metadata/ServiceInstanceMetadataUtilsTest.java |   4 +-
 .../registry/nacos/NacosServiceDiscovery.java      |  10 +-
 .../zookeeper/ZookeeperServiceDiscovery.java       |  22 +-
 .../ZookeeperServiceDiscoveryChangeWatcher.java    |   2 +-
 .../java/org/apache/dubbo/rpc/RpcException.java    |   1 +
 .../apache/dubbo/rpc/filter/ActiveLimitFilter.java | 244 +++++------
 .../apache/dubbo/rpc/filter/ClassLoaderFilter.java |  88 ++--
 .../apache/dubbo/rpc/filter/CompatibleFilter.java  | 184 ++++-----
 .../apache/dubbo/rpc/filter/DeprecatedFilter.java  | 156 +++----
 .../org/apache/dubbo/rpc/filter/EchoFilter.java    |  88 ++--
 .../apache/dubbo/rpc/filter/ExceptionFilter.java   | 230 +++++------
 .../dubbo/rpc/filter/ExecuteLimitFilter.java       | 172 ++++----
 .../apache/dubbo/rpc/filter/GenericImplFilter.java | 458 ++++++++++-----------
 .../org/apache/dubbo/rpc/filter/TimeoutFilter.java | 138 +++----
 .../org/apache/dubbo/rpc/filter/TokenFilter.java   | 114 ++---
 .../rpc/protocol/ProtocolListenerWrapper.java      |   4 +-
 .../rpc/support/DefaultFilterChainBuilder.java     | 125 ------
 .../dubbo/internal/org.apache.dubbo.rpc.Filter     |  28 +-
 .../org.apache.dubbo.rpc.FilterChainBuilder        |   1 -
 .../dubbo/internal/org.apache.dubbo.rpc.Protocol   |   1 -
 .../dubbo/rpc/filter/AccessLogFilterTest.java      | 168 ++++----
 .../dubbo/rpc/filter/ActiveLimitFilterTest.java    | 452 ++++++++++----------
 .../rpc/filter/CompatibleFilterFilterTest.java     | 346 ++++++++--------
 .../apache/dubbo/rpc/filter/ContextFilterTest.java | 150 +++----
 .../dubbo/rpc/filter/DeprecatedFilterTest.java     |  95 +++--
 .../apache/dubbo/rpc/filter/EchoFilterTest.java    | 158 +++----
 .../dubbo/rpc/filter/ExceptionFilterTest.java      | 276 ++++++-------
 103 files changed, 3464 insertions(+), 2544 deletions(-)

diff --git a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/DefaultFilterChainBuilder.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/DefaultFilterChainBuilder.java
new file mode 100644
index 0000000..982008b
--- /dev/null
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/DefaultFilterChainBuilder.java
@@ -0,0 +1,62 @@
+/*
+ * 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.rpc.cluster.filter;
+
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.extension.ExtensionLoader;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.cluster.ClusterInvoker;
+
+import java.util.List;
+
+@Activate(order = 0)
+public class DefaultFilterChainBuilder implements FilterChainBuilder {
+
+    @Override
+    public <T> Invoker<T> buildInvokerChain(final Invoker<T> originalInvoker, String key, String group) {
+        Invoker<T> last = originalInvoker;
+        List<Filter> filters = ExtensionLoader.getExtensionLoader(Filter.class).getActivateExtension(originalInvoker.getUrl(), key, group);
+
+        if (!filters.isEmpty()) {
+            for (int i = filters.size() - 1; i >= 0; i--) {
+                final Filter filter = filters.get(i);
+                final Invoker<T> next = last;
+                last = new FilterChainNode<>(originalInvoker, next, filter);
+            }
+        }
+
+        return last;
+    }
+
+    @Override
+    public <T> ClusterInvoker<T> buildClusterInvokerChain(final ClusterInvoker<T> originalInvoker, String key, String group) {
+        ClusterInvoker<T> last = originalInvoker;
+        List<Filter> filters = ExtensionLoader.getExtensionLoader(Filter.class).getActivateExtension(originalInvoker.getUrl(), key, group);
+
+        if (!filters.isEmpty()) {
+            for (int i = filters.size() - 1; i >= 0; i--) {
+                final Filter filter = filters.get(i);
+                final Invoker<T> next = last;
+                last = new ClusterFilterChainNode<>(originalInvoker, next, filter);
+            }
+        }
+
+        return last;
+    }
+
+}
diff --git a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/FilterChainBuilder.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/FilterChainBuilder.java
new file mode 100644
index 0000000..20275e2
--- /dev/null
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/FilterChainBuilder.java
@@ -0,0 +1,148 @@
+/*
+ * 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.rpc.cluster.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.extension.SPI;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.ListenableFilter;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.cluster.ClusterInvoker;
+import org.apache.dubbo.rpc.cluster.Directory;
+
+@SPI("default")
+public interface FilterChainBuilder {
+    <T> Invoker<T> buildInvokerChain(final Invoker<T> invoker, String key, String group);
+
+    <T> ClusterInvoker<T> buildClusterInvokerChain(final ClusterInvoker<T> invoker, String key, String group);
+
+    class FilterChainNode<T, TYPE extends Invoker<T>> implements Invoker<T>{
+        TYPE originalInvoker;
+        Invoker<T> nextNode;
+        Filter filter;
+
+        public FilterChainNode(TYPE originalInvoker, Invoker<T> nextNode, Filter filter) {
+            this.originalInvoker = originalInvoker;
+            this.nextNode = nextNode;
+            this.filter = filter;
+        }
+
+        public TYPE getOriginalInvoker() {
+            return originalInvoker;
+        }
+
+        @Override
+        public Class<T> getInterface() {
+            return originalInvoker.getInterface();
+        }
+
+        @Override
+        public URL getUrl() {
+            return originalInvoker.getUrl();
+        }
+
+        @Override
+        public boolean isAvailable() {
+            return originalInvoker.isAvailable();
+        }
+
+        @Override
+        public Result invoke(Invocation invocation) throws RpcException {
+            Result asyncResult;
+            try {
+                asyncResult = filter.invoke(nextNode, invocation);
+            } catch (Exception e) {
+                if (filter instanceof ListenableFilter) {
+                    ListenableFilter listenableFilter = ((ListenableFilter) filter);
+                    try {
+                        Filter.Listener listener = listenableFilter.listener(invocation);
+                        if (listener != null) {
+                            listener.onError(e, originalInvoker, invocation);
+                        }
+                    } finally {
+                        listenableFilter.removeListener(invocation);
+                    }
+                } else if (filter instanceof Filter.Listener) {
+                    Filter.Listener listener = (Filter.Listener) filter;
+                    listener.onError(e, originalInvoker, invocation);
+                }
+                throw e;
+            } finally {
+
+            }
+            return asyncResult.whenCompleteWithContext((r, t) -> {
+                if (filter instanceof ListenableFilter) {
+                    ListenableFilter listenableFilter = ((ListenableFilter) filter);
+                    Filter.Listener listener = listenableFilter.listener(invocation);
+                    try {
+                        if (listener != null) {
+                            if (t == null) {
+                                listener.onResponse(r, originalInvoker, invocation);
+                            } else {
+                                listener.onError(t, originalInvoker, invocation);
+                            }
+                        }
+                    } finally {
+                        listenableFilter.removeListener(invocation);
+                    }
+                } else if (filter instanceof Filter.Listener) {
+                    Filter.Listener listener = (Filter.Listener) filter;
+                    if (t == null) {
+                        listener.onResponse(r, originalInvoker, invocation);
+                    } else {
+                        listener.onError(t, originalInvoker, invocation);
+                    }
+                }
+            });
+        }
+
+        @Override
+        public void destroy() {
+            originalInvoker.destroy();
+        }
+
+        @Override
+        public String toString() {
+            return originalInvoker.toString();
+        }
+    }
+
+    class ClusterFilterChainNode<T, TYPE extends ClusterInvoker<T>> extends FilterChainNode<T, TYPE> implements ClusterInvoker<T> {
+        public ClusterFilterChainNode(TYPE originalInvoker, Invoker<T> nextNode, Filter filter) {
+            super(originalInvoker, nextNode, filter);
+        }
+
+
+        @Override
+        public URL getRegistryUrl() {
+            return getOriginalInvoker().getRegistryUrl();
+        }
+
+        @Override
+        public Directory<T> getDirectory() {
+            return getOriginalInvoker().getDirectory();
+        }
+
+        @Override
+        public boolean isDestroyed() {
+            return getOriginalInvoker().isDestroyed();
+        }
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/InvocationInterceptorBuilder.java
similarity index 76%
copy from dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java
copy to dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/InvocationInterceptorBuilder.java
index b8a7ac5..6f737a4 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/InvocationInterceptorBuilder.java
@@ -14,11 +14,12 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dubbo.rpc;
+package org.apache.dubbo.rpc.cluster.filter;
 
 import org.apache.dubbo.common.extension.SPI;
+import org.apache.dubbo.rpc.cluster.ClusterInvoker;
 
 @SPI("default")
-public interface FilterChainBuilder {
-    <T> Invoker<T> buildInvokerChain(final Invoker<T> invoker, String key, String group);
+public interface InvocationInterceptorBuilder {
+    <T> ClusterInvoker<T> buildClusterInterceptorChain(final ClusterInvoker<T> invoker, String key, String group);
 }
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolFilterWrapper.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/ProtocolFilterWrapper.java
similarity index 94%
rename from dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolFilterWrapper.java
rename to dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/ProtocolFilterWrapper.java
index 24fcc8a..cded6dd 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolFilterWrapper.java
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/filter/ProtocolFilterWrapper.java
@@ -1,83 +1,82 @@
-/*
- * 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.rpc.protocol;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.extension.ExtensionLoader;
-import org.apache.dubbo.common.utils.UrlUtils;
-import org.apache.dubbo.rpc.Exporter;
-import org.apache.dubbo.rpc.FilterChainBuilder;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Protocol;
-import org.apache.dubbo.rpc.ProtocolServer;
-import org.apache.dubbo.rpc.RpcException;
-
-import java.util.List;
-
-import static org.apache.dubbo.common.constants.CommonConstants.SERVICE_FILTER_KEY;
-
-/**
- * ListenerProtocol
- */
-@Activate(order = 100)
-public class ProtocolFilterWrapper implements Protocol {
-
-    private final Protocol protocol;
-    private static final FilterChainBuilder builder
-            = ExtensionLoader.getExtensionLoader(FilterChainBuilder.class).getDefaultExtension();
-
-    public ProtocolFilterWrapper(Protocol protocol) {
-        if (protocol == null) {
-            throw new IllegalArgumentException("protocol == null");
-        }
-        this.protocol = protocol;
-    }
-
-    @Override
-    public int getDefaultPort() {
-        return protocol.getDefaultPort();
-    }
-
-    @Override
-    public <T> Exporter<T> export(Invoker<T> invoker) throws RpcException {
-        if (UrlUtils.isRegistry(invoker.getUrl())) {
-            return protocol.export(invoker);
-        }
-        return protocol.export(builder.buildInvokerChain(invoker, SERVICE_FILTER_KEY, CommonConstants.PROVIDER));
-    }
-
-    @Override
-    public <T> Invoker<T> refer(Class<T> type, URL url) throws RpcException {
-        if (UrlUtils.isRegistry(url)) {
-            return protocol.refer(type, url);
-        }
-        return protocol.refer(type, url);
-    }
-
-    @Override
-    public void destroy() {
-        protocol.destroy();
-    }
-
-    @Override
-    public List<ProtocolServer> getServers() {
-        return protocol.getServers();
-    }
-
-}
+/*
+ * 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.rpc.cluster.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.extension.ExtensionLoader;
+import org.apache.dubbo.common.utils.UrlUtils;
+import org.apache.dubbo.rpc.Exporter;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Protocol;
+import org.apache.dubbo.rpc.ProtocolServer;
+import org.apache.dubbo.rpc.RpcException;
+
+import java.util.List;
+
+import static org.apache.dubbo.common.constants.CommonConstants.SERVICE_FILTER_KEY;
+
+/**
+ * ListenerProtocol
+ */
+@Activate(order = 100)
+public class ProtocolFilterWrapper implements Protocol {
+
+    private final Protocol protocol;
+    private static final FilterChainBuilder builder
+            = ExtensionLoader.getExtensionLoader(FilterChainBuilder.class).getDefaultExtension();
+
+    public ProtocolFilterWrapper(Protocol protocol) {
+        if (protocol == null) {
+            throw new IllegalArgumentException("protocol == null");
+        }
+        this.protocol = protocol;
+    }
+
+    @Override
+    public int getDefaultPort() {
+        return protocol.getDefaultPort();
+    }
+
+    @Override
+    public <T> Exporter<T> export(Invoker<T> invoker) throws RpcException {
+        if (UrlUtils.isRegistry(invoker.getUrl())) {
+            return protocol.export(invoker);
+        }
+        return protocol.export(builder.buildInvokerChain(invoker, SERVICE_FILTER_KEY, CommonConstants.PROVIDER));
+    }
+
+    @Override
+    public <T> Invoker<T> refer(Class<T> type, URL url) throws RpcException {
+        if (UrlUtils.isRegistry(url)) {
+            return protocol.refer(type, url);
+        }
+        return protocol.refer(type, url);
+    }
+
+    @Override
+    public void destroy() {
+        protocol.destroy();
+    }
+
+    @Override
+    public List<ProtocolServer> getServers() {
+        return protocol.getServers();
+    }
+
+}
diff --git a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/AbstractClusterInvoker.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/AbstractClusterInvoker.java
index acd150f..381d894 100644
--- a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/AbstractClusterInvoker.java
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/AbstractClusterInvoker.java
@@ -81,16 +81,16 @@ public abstract class AbstractClusterInvoker<T> implements ClusterInvoker<T> {
 
     @Override
     public Class<T> getInterface() {
-        return directory.getInterface();
+        return getDirectory().getInterface();
     }
 
     @Override
     public URL getUrl() {
-        return directory.getConsumerUrl();
+        return getDirectory().getConsumerUrl();
     }
 
     public URL getRegistryUrl() {
-        return directory.getUrl();
+        return getDirectory().getUrl();
     }
 
     @Override
@@ -99,7 +99,7 @@ public abstract class AbstractClusterInvoker<T> implements ClusterInvoker<T> {
         if (invoker != null) {
             return invoker.isAvailable();
         }
-        return directory.isAvailable();
+        return getDirectory().isAvailable();
     }
 
     public Directory<T> getDirectory() {
@@ -109,7 +109,7 @@ public abstract class AbstractClusterInvoker<T> implements ClusterInvoker<T> {
     @Override
     public void destroy() {
         if (destroyed.compareAndSet(false, true)) {
-            directory.destroy();
+            getDirectory().destroy();
         }
     }
 
@@ -284,8 +284,8 @@ public abstract class AbstractClusterInvoker<T> implements ClusterInvoker<T> {
         if (CollectionUtils.isEmpty(invokers)) {
             throw new RpcException(RpcException.NO_INVOKER_AVAILABLE_AFTER_FILTER, "Failed to invoke the method "
                     + invocation.getMethodName() + " in the service " + getInterface().getName()
-                    + ". No provider available for the service " + directory.getConsumerUrl().getServiceKey()
-                    + " from registry " + directory.getUrl().getAddress()
+                    + ". No provider available for the service " + getDirectory().getConsumerUrl().getServiceKey()
+                    + " from registry " + getDirectory().getUrl().getAddress()
                     + " on the consumer " + NetUtils.getLocalHost()
                     + " using the dubbo version " + Version.getVersion()
                     + ". Please check if the providers have been started and registered.");
@@ -307,7 +307,7 @@ public abstract class AbstractClusterInvoker<T> implements ClusterInvoker<T> {
                                        LoadBalance loadbalance) throws RpcException;
 
     protected List<Invoker<T>> list(Invocation invocation) throws RpcException {
-        return directory.list(invocation);
+        return getDirectory().list(invocation);
     }
 
     /**
diff --git a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/wrapper/AbstractCluster.java b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/wrapper/AbstractCluster.java
index 6bdfeb7..9ce920d 100644
--- a/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/wrapper/AbstractCluster.java
+++ b/dubbo-cluster/src/main/java/org/apache/dubbo/rpc/cluster/support/wrapper/AbstractCluster.java
@@ -19,21 +19,24 @@ package org.apache.dubbo.rpc.cluster.support.wrapper;
 import org.apache.dubbo.common.URL;
 import org.apache.dubbo.common.constants.CommonConstants;
 import org.apache.dubbo.common.extension.ExtensionLoader;
-import org.apache.dubbo.rpc.FilterChainBuilder;
+import org.apache.dubbo.common.utils.CollectionUtils;
 import org.apache.dubbo.rpc.Invocation;
 import org.apache.dubbo.rpc.Invoker;
 import org.apache.dubbo.rpc.Result;
 import org.apache.dubbo.rpc.RpcException;
 import org.apache.dubbo.rpc.cluster.Cluster;
+import org.apache.dubbo.rpc.cluster.ClusterInvoker;
 import org.apache.dubbo.rpc.cluster.Directory;
 import org.apache.dubbo.rpc.cluster.LoadBalance;
+import org.apache.dubbo.rpc.cluster.directory.StaticDirectory;
+import org.apache.dubbo.rpc.cluster.filter.FilterChainBuilder;
+import org.apache.dubbo.rpc.cluster.filter.InvocationInterceptorBuilder;
 import org.apache.dubbo.rpc.cluster.interceptor.ClusterInterceptor;
 import org.apache.dubbo.rpc.cluster.support.AbstractClusterInvoker;
 
 import java.util.List;
 
-import static org.apache.dubbo.common.constants.CommonConstants.DEFAULT_KEY;
-import static org.apache.dubbo.common.constants.CommonConstants.FILTER_BUILDER_KEY;
+import static org.apache.dubbo.common.constants.CommonConstants.INVOCATION_INTERCEPTOR_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.REFERENCE_FILTER_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.REFERENCE_INTERCEPTOR_KEY;
 
@@ -41,8 +44,8 @@ public abstract class AbstractCluster implements Cluster {
 
     private <T> Invoker<T> buildClusterInterceptors(AbstractClusterInvoker<T> clusterInvoker, String key) {
 //        AbstractClusterInvoker<T> last = clusterInvoker;
-        AbstractClusterInvoker<T> last = new FilterInvoker<>(clusterInvoker);
-        List<ClusterInterceptor> interceptors = ExtensionLoader.getExtensionLoader(ClusterInterceptor.class).getActivateExtension(clusterInvoker.getUrl(), key);
+        AbstractClusterInvoker<T> last = buildInterceptorInvoker(new FilterInvoker<>(clusterInvoker));
+        List<ClusterInterceptor> interceptors = ExtensionLoader.getExtensionLoader(ClusterInterceptor.class).getActivateExtensions();
 
         if (!interceptors.isEmpty()) {
             for (int i = interceptors.size() - 1; i >= 0; i--) {
@@ -56,12 +59,23 @@ public abstract class AbstractCluster implements Cluster {
 
     @Override
     public <T> Invoker<T> join(Directory<T> directory) throws RpcException {
+        if (directory instanceof StaticDirectory) {
+            return doJoin(directory);
+        }
         return buildClusterInterceptors(doJoin(directory), directory.getUrl().getParameter(REFERENCE_INTERCEPTOR_KEY));
     }
 
+    private <T> AbstractClusterInvoker<T> buildInterceptorInvoker(AbstractClusterInvoker<T> invoker) {
+        List<InvocationInterceptorBuilder> builders = ExtensionLoader.getExtensionLoader(InvocationInterceptorBuilder.class).getActivateExtensions();
+        if (CollectionUtils.isEmpty(builders)) {
+            return invoker;
+        }
+        return new InterceptorInvoker<>(invoker, builders);
+    }
+
     protected abstract <T> AbstractClusterInvoker<T> doJoin(Directory<T> directory) throws RpcException;
 
-    protected class InterceptorInvokerNode<T> extends AbstractClusterInvoker<T> {
+    static class InterceptorInvokerNode<T> extends AbstractClusterInvoker<T> {
 
         private AbstractClusterInvoker<T> clusterInvoker;
         private ClusterInterceptor interceptor;
@@ -137,12 +151,19 @@ public abstract class AbstractCluster implements Cluster {
     }
 
     static class FilterInvoker<T> extends AbstractClusterInvoker<T> {
-        private Invoker<T> filterInvoker;
+        private ClusterInvoker<T> filterInvoker;
 
         public FilterInvoker(AbstractClusterInvoker<T> invoker) {
-            filterInvoker = ExtensionLoader.getExtensionLoader(FilterChainBuilder.class)
-                    .getExtension(invoker.getUrl().getParameter(FILTER_BUILDER_KEY, DEFAULT_KEY))
-                    .buildInvokerChain(invoker, REFERENCE_FILTER_KEY, CommonConstants.CONSUMER);
+            List<FilterChainBuilder> builders = ExtensionLoader.getExtensionLoader(FilterChainBuilder.class).getActivateExtensions();
+            if (CollectionUtils.isEmpty(builders)) {
+                filterInvoker = invoker;
+            } else {
+                ClusterInvoker<T> tmpInvoker = invoker;
+                for (FilterChainBuilder builder : builders) {
+                    tmpInvoker = builder.buildClusterInvokerChain(tmpInvoker, REFERENCE_FILTER_KEY, CommonConstants.CONSUMER);
+                }
+                filterInvoker = tmpInvoker;
+            }
         }
 
         @Override
@@ -150,6 +171,28 @@ public abstract class AbstractCluster implements Cluster {
             return filterInvoker.invoke(invocation);
         }
 
+        @Override
+        public Directory<T> getDirectory() {
+            return filterInvoker.getDirectory();
+        }
+
+
+
+        @Override
+        public URL getRegistryUrl() {
+            return filterInvoker.getRegistryUrl();
+        }
+
+        @Override
+        public boolean isDestroyed() {
+            return filterInvoker.isDestroyed();
+        }
+
+        @Override
+        public URL getUrl() {
+            return filterInvoker.getUrl();
+        }
+
         /**
          * The only purpose is to build a interceptor chain, so the cluster related logic doesn't matter.
          * Use ClusterInvoker<T> to replace AbstractClusterInvoker<T> in the future.
@@ -160,5 +203,49 @@ public abstract class AbstractCluster implements Cluster {
         }
     }
 
+    static class InterceptorInvoker<T> extends AbstractClusterInvoker<T> {
+        private ClusterInvoker<T> interceptorInvoker;
+
+        public InterceptorInvoker(AbstractClusterInvoker<T> invoker, List<InvocationInterceptorBuilder> builders) {
+            ClusterInvoker<T> tmpInvoker = invoker;
+            for (InvocationInterceptorBuilder builder : builders) {
+                tmpInvoker = builder.buildClusterInterceptorChain(tmpInvoker, INVOCATION_INTERCEPTOR_KEY, CommonConstants.CONSUMER);
+            }
+            interceptorInvoker = tmpInvoker;
+        }
+
+        @Override
+        public Result invoke(Invocation invocation) throws RpcException {
+            return interceptorInvoker.invoke(invocation);
+        }
 
+        @Override
+        public Directory<T> getDirectory() {
+            return interceptorInvoker.getDirectory();
+        }
+
+        @Override
+        public URL getRegistryUrl() {
+            return interceptorInvoker.getRegistryUrl();
+        }
+
+        @Override
+        public boolean isDestroyed() {
+            return interceptorInvoker.isDestroyed();
+        }
+
+        @Override
+        public URL getUrl() {
+            return interceptorInvoker.getUrl();
+        }
+
+        /**
+         * The only purpose is to build a interceptor chain, so the cluster related logic doesn't matter.
+         * Use ClusterInvoker<T> to replace AbstractClusterInvoker<T> in the future.
+         */
+        @Override
+        protected Result doInvoke(Invocation invocation, List<Invoker<T>> invokers, LoadBalance loadbalance) throws RpcException {
+            return null;
+        }
+    }
 }
diff --git a/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol b/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol
new file mode 100644
index 0000000..9a153fd
--- /dev/null
+++ b/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol
@@ -0,0 +1 @@
+filter=org.apache.dubbo.rpc.cluster.filter.ProtocolFilterWrapper
\ No newline at end of file
diff --git a/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.cluster.filter.FilterChainBuilder b/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.cluster.filter.FilterChainBuilder
new file mode 100644
index 0000000..ea1c452
--- /dev/null
+++ b/dubbo-cluster/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.cluster.filter.FilterChainBuilder
@@ -0,0 +1 @@
+default=org.apache.dubbo.rpc.cluster.filter.DefaultFilterChainBuilder
\ No newline at end of file
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/URL.java b/dubbo-common/src/main/java/org/apache/dubbo/common/URL.java
index 460b8ed..485ce7c 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/URL.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/URL.java
@@ -48,6 +48,7 @@ import java.util.TreeMap;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.function.Predicate;
 
+import static org.apache.dubbo.common.BaseServiceMetadata.COLON_SEPERATOR;
 import static org.apache.dubbo.common.constants.CommonConstants.ANYHOST_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.ANYHOST_VALUE;
 import static org.apache.dubbo.common.constants.CommonConstants.APPLICATION_KEY;
@@ -1297,6 +1298,19 @@ class URL implements Serializable {
     }
 
     /**
+     * Format : interface:version
+     *
+     * @return
+     */
+    public String getDisplayServiceKey() {
+        if (StringUtils.isEmpty(getVersion())) {
+            return getServiceInterface();
+        }
+        return getServiceInterface() +
+                COLON_SEPERATOR + getVersion();
+    }
+
+    /**
      * The format of return value is '{group}/{path/interfaceName}:{version}'
      *
      * @return
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/constants/CommonConstants.java b/dubbo-common/src/main/java/org/apache/dubbo/common/constants/CommonConstants.java
index 5bb4e5f..4f831b4 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/constants/CommonConstants.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/constants/CommonConstants.java
@@ -263,6 +263,8 @@ public interface CommonConstants {
 
     String REFERENCE_FILTER_KEY = "reference.filter";
 
+    String INVOCATION_INTERCEPTOR_KEY = "invocation.interceptor";
+
     String INVOKER_LISTENER_KEY = "invoker.listener";
 
     String DUBBO_VERSION_KEY = "dubbo";
@@ -363,8 +365,6 @@ public interface CommonConstants {
 
     String REGISTRY_DELAY_NOTIFICATION_KEY = "delay-notification";
 
-    String FILTER_BUILDER_KEY = "filter-builder";
-
     String CACHE_CLEAR_TASK_INTERVAL = "dubbo.application.url.cache.task.interval";
     String CACHE_CLEAR_WAITING_THRESHOLD = "dubbo.application.url.cache.clear.waiting";
 
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/constants/RegistryConstants.java b/dubbo-common/src/main/java/org/apache/dubbo/common/constants/RegistryConstants.java
index f70304d..bd8f43c 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/constants/RegistryConstants.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/constants/RegistryConstants.java
@@ -23,8 +23,6 @@ public interface RegistryConstants {
 
     String REGISTRY_CLUSTER_KEY = "REGISTRY_CLUSTER";
 
-    String REGISTRY_CLUSTER = "REGISTRY_CLUSTER";
-
     String REGISTRY_CLUSTER_TYPE_KEY = "registry-cluster-type";
 
     String REGISTRY_PROTOCOL = "registry";
@@ -63,7 +61,11 @@ public interface RegistryConstants {
 
     String REGISTRY_PUBLISH_INTERFACE_KEY = "publish-interface";
 
+    String REGISTRY_PUBLISH_INSTANCE_KEY = "publish-instance";
+
     String DUBBO_PUBLISH_INTERFACE_DEFAULT_KEY = "dubbo.application.publish-interface";
+
+    String DUBBO_PUBLISH_INSTANCE_DEFAULT_KEY = "dubbo.application.publish-instance";
     /**
      * The parameter key of Dubbo Registry type
      *
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/extension/Activate.java b/dubbo-common/src/main/java/org/apache/dubbo/common/extension/Activate.java
index ba1873e..7793500 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/extension/Activate.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/extension/Activate.java
@@ -86,6 +86,8 @@ public @interface Activate {
     /**
      * Absolute ordering info, optional
      *
+     * Ascending order, smaller values will be in the front o the list.
+     *
      * @return absolute ordering info
      */
     int order() default 0;
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/extension/ExtensionLoader.java b/dubbo-common/src/main/java/org/apache/dubbo/common/extension/ExtensionLoader.java
index 8b88422..26489e0 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/extension/ExtensionLoader.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/extension/ExtensionLoader.java
@@ -307,6 +307,21 @@ public class ExtensionLoader<T> {
         return activateExtensions;
     }
 
+    public List<T> getActivateExtensions() {
+        List<T> activateExtensions = new ArrayList<>();
+        getExtensionClasses();
+        for (Map.Entry<String, Object> entry : cachedActivates.entrySet()) {
+            String name = entry.getKey();
+            Object activate = entry.getValue();
+            if (!(activate instanceof Activate)) {
+                continue;
+            }
+            activateExtensions.add(getExtension(name));
+        }
+        activateExtensions.sort(ActivateComparator.COMPARATOR);
+        return activateExtensions;
+    }
+
     private boolean isMatchGroup(String group, String[] groups) {
         if (StringUtils.isEmpty(group)) {
             return true;
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/common/threadpool/manager/DefaultExecutorRepository.java b/dubbo-common/src/main/java/org/apache/dubbo/common/threadpool/manager/DefaultExecutorRepository.java
index 398f350..b8eb41e 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/common/threadpool/manager/DefaultExecutorRepository.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/common/threadpool/manager/DefaultExecutorRepository.java
@@ -173,7 +173,7 @@ public class DefaultExecutorRepository implements ExecutorRepository {
 
     @Override
     public ScheduledExecutorService getMetadataRetryExecutor() {
-        return null;
+        return metadataRetryExecutor;
     }
 
     @Override
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractConfig.java b/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractConfig.java
index db2b583..4e7bbf4 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractConfig.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractConfig.java
@@ -461,6 +461,7 @@ public abstract class AbstractConfig implements Serializable {
     }
 
     public void refresh() {
+        refreshed.set(true);
         Environment env = ApplicationModel.getEnvironment();
         try {
             CompositeConfiguration compositeConfiguration = env.getPrefixedConfiguration(this);
@@ -494,6 +495,11 @@ public abstract class AbstractConfig implements Serializable {
         }
     }
 
+    @Parameter(excluded = true)
+    public boolean isRefreshed() {
+        return refreshed.get();
+    }
+
     @Override
     public String toString() {
         try {
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractInterfaceConfig.java b/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractInterfaceConfig.java
index cbab17a..4be28bf 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractInterfaceConfig.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/config/AbstractInterfaceConfig.java
@@ -455,14 +455,10 @@ public abstract class AbstractInterfaceConfig extends AbstractMethodConfig {
      * Always use the global ApplicationConfig
      */
     public ApplicationConfig getApplication() {
-        ApplicationConfig globalApplication = ApplicationModel.getConfigManager().getApplicationOrElseThrow();
-        if (globalApplication == null) {
-            return application;
-        }
-        if (application != null && !application.getName().equals(globalApplication.getName())) {
+        if (application != null) {
             return application;
         }
-        return globalApplication;
+        return ApplicationModel.getConfigManager().getApplicationOrElseThrow();
     }
 
     @Deprecated
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/config/ApplicationConfig.java b/dubbo-common/src/main/java/org/apache/dubbo/config/ApplicationConfig.java
index e8d79ee..f77d3f1 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/config/ApplicationConfig.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/config/ApplicationConfig.java
@@ -34,6 +34,7 @@ import java.util.Map;
 import java.util.Set;
 
 import static org.apache.dubbo.common.constants.CommonConstants.APPLICATION_KEY;
+import static org.apache.dubbo.common.constants.CommonConstants.DUBBO;
 import static org.apache.dubbo.common.constants.CommonConstants.DUMP_DIRECTORY;
 import static org.apache.dubbo.common.constants.CommonConstants.HOST_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.METADATA_KEY;
@@ -165,6 +166,12 @@ public class ApplicationConfig extends AbstractConfig {
 
     private Boolean publishInterface;
 
+    /**
+     * The preferred protocol(name) of this application
+     * convenient for places where it's hard to determine which is the preferred protocol
+     */
+    private String protocol;
+
     public ApplicationConfig() {
     }
 
@@ -469,6 +476,15 @@ public class ApplicationConfig extends AbstractConfig {
         this.publishInterface = publishInterface;
     }
 
+    @Parameter(excluded = true, key="application-protocol")
+    public String getProtocol() {
+        return protocol == null ? DUBBO : protocol;
+    }
+
+    public void setProtocol(String protocol) {
+        this.protocol = protocol;
+    }
+
     @Override
     public void refresh() {
         super.refresh();
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/config/RegistryConfig.java b/dubbo-common/src/main/java/org/apache/dubbo/config/RegistryConfig.java
index 817bbc7..2999682 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/config/RegistryConfig.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/config/RegistryConfig.java
@@ -25,6 +25,8 @@ import java.util.Map;
 
 import static org.apache.dubbo.common.constants.CommonConstants.EXTRA_KEYS_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.SHUTDOWN_WAIT_KEY;
+import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_PUBLISH_INSTANCE_KEY;
+import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_PUBLISH_INTERFACE_KEY;
 import static org.apache.dubbo.common.constants.RemotingConstants.BACKUP_KEY;
 import static org.apache.dubbo.common.utils.PojoUtils.updatePropertyIfAbsent;
 import static org.apache.dubbo.config.Constants.REGISTRIES_SUFFIX;
@@ -182,6 +184,10 @@ public class RegistryConfig extends AbstractConfig {
      */
     private Integer weight;
 
+    private Boolean publishInterface;
+
+    private Boolean publishInstance;
+
     public RegistryConfig() {
     }
 
@@ -515,6 +521,24 @@ public class RegistryConfig extends AbstractConfig {
         this.weight = weight;
     }
 
+    @Parameter(key = REGISTRY_PUBLISH_INTERFACE_KEY)
+    public Boolean getPublishInterface() {
+        return publishInterface;
+    }
+
+    public void setPublishInterface(Boolean publishInterface) {
+        this.publishInterface = publishInterface;
+    }
+
+    @Parameter(key = REGISTRY_PUBLISH_INSTANCE_KEY)
+    public Boolean getPublishInstance() {
+        return publishInstance;
+    }
+
+    public void setPublishInstance(Boolean publishInstance) {
+        this.publishInstance = publishInstance;
+    }
+
     @Override
     public void refresh() {
         super.refresh();
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/ServiceConfig.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/ServiceConfig.java
index e2e8b09..3b5edc1 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/ServiceConfig.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/ServiceConfig.java
@@ -37,6 +37,7 @@ import org.apache.dubbo.config.support.Parameter;
 import org.apache.dubbo.config.utils.ConfigValidationUtils;
 import org.apache.dubbo.event.Event;
 import org.apache.dubbo.event.EventDispatcher;
+import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.ServiceNameMapping;
 import org.apache.dubbo.registry.client.metadata.MetadataUtils;
 import org.apache.dubbo.rpc.Exporter;
@@ -211,8 +212,6 @@ public class ServiceConfig<T> extends ServiceConfigBase<T> {
         } else {
             doExport();
         }
-
-        exported();
     }
 
     public void exported() {
@@ -308,6 +307,7 @@ public class ServiceConfig<T> extends ServiceConfigBase<T> {
             path = interfaceName;
         }
         doExportUrls();
+        exported();
     }
 
     @SuppressWarnings({"unchecked", "rawtypes"})
@@ -506,6 +506,9 @@ public class ServiceConfig<T> extends ServiceConfigBase<T> {
                     if (logger.isInfoEnabled()) {
                         logger.info("Export dubbo service " + interfaceClass.getName() + " to url " + url);
                     }
+                    if (MetadataService.class.getName().equals(url.getServiceInterface())) {
+                        MetadataUtils.saveMetadataURL(url);
+                    }
                     Invoker<?> invoker = PROXY_FACTORY.getInvoker(ref, (Class) interfaceClass, url);
                     DelegateProviderMetaDataInvoker wrapperInvoker = new DelegateProviderMetaDataInvoker(invoker, this);
 
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/bootstrap/DubboBootstrap.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/bootstrap/DubboBootstrap.java
index e74a5e4..48bc281 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/bootstrap/DubboBootstrap.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/bootstrap/DubboBootstrap.java
@@ -68,7 +68,6 @@ import org.apache.dubbo.metadata.report.MetadataReportFactory;
 import org.apache.dubbo.metadata.report.MetadataReportInstance;
 import org.apache.dubbo.registry.client.DefaultServiceInstance;
 import org.apache.dubbo.registry.client.ServiceInstance;
-import org.apache.dubbo.registry.client.ServiceInstanceCustomizer;
 import org.apache.dubbo.registry.client.metadata.MetadataUtils;
 import org.apache.dubbo.registry.client.metadata.ServiceInstanceMetadataUtils;
 import org.apache.dubbo.registry.client.metadata.store.InMemoryWritableMetadataService;
@@ -82,7 +81,6 @@ import java.util.Collection;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Set;
-import java.util.SortedSet;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Future;
@@ -862,7 +860,7 @@ public class DubboBootstrap extends GenericEventListener {
      * Initialize {@link MetadataService} from {@link WritableMetadataService}'s extension
      */
     private void initMetadataService() {
-        startMetadataCenter();
+//        startMetadataCenter();
         this.metadataService = getDefaultExtension();
         this.metadataServiceExporter = new ConfigurableMetadataServiceExporter(metadataService);
     }
@@ -1140,21 +1138,11 @@ public class DubboBootstrap extends GenericEventListener {
     }
 
     private void registerServiceInstance() {
-        if (CollectionUtils.isEmpty(getServiceDiscoveries())) {
-            return;
-        }
-
         ApplicationConfig application = getApplication();
 
         String serviceName = application.getName();
 
-        URL exportedURL = selectMetadataServiceExportedURL();
-
-        String host = exportedURL.getHost();
-
-        int port = exportedURL.getPort();
-
-        ServiceInstance serviceInstance = createServiceInstance(serviceName, host, port);
+        ServiceInstance serviceInstance = createServiceInstance(serviceName);
 
         doRegisterServiceInstance(serviceInstance);
 
@@ -1162,62 +1150,37 @@ public class DubboBootstrap extends GenericEventListener {
         executorRepository.nextScheduledExecutor().scheduleAtFixedRate(() -> {
             InMemoryWritableMetadataService localMetadataService = (InMemoryWritableMetadataService) WritableMetadataService.getDefaultExtension();
             localMetadataService.blockUntilUpdated();
-            ServiceInstanceMetadataUtils.refreshMetadataAndInstance();
-        }, 0, ConfigurationUtils.get(METADATA_PUBLISH_DELAY_KEY, DEFAULT_METADATA_PUBLISH_DELAY), TimeUnit.MICROSECONDS);
+            ServiceInstanceMetadataUtils.refreshMetadataAndInstance(serviceInstance);
+        }, 0, ConfigurationUtils.get(METADATA_PUBLISH_DELAY_KEY, DEFAULT_METADATA_PUBLISH_DELAY), TimeUnit.MILLISECONDS);
     }
 
     private void doRegisterServiceInstance(ServiceInstance serviceInstance) {
-        //FIXME
-        if (logger.isInfoEnabled()) {
-            logger.info("Start publishing metadata to remote center, this only makes sense for applications enabled remote metadata center.");
+        // register instance only when at least one service is exported.
+        if (serviceInstance.getPort() != null && serviceInstance.getPort() != -1) {
+            publishMetadataToRemote(serviceInstance);
+            logger.info("Start registering instance address to registry.");
+            getServiceDiscoveries().forEach(serviceDiscovery ->
+            {
+                calInstanceRevision(serviceDiscovery, serviceInstance);
+                if (logger.isDebugEnabled()) {
+                    logger.info("Start registering instance address to registry" + serviceDiscovery.getUrl() + ", instance " + serviceInstance);
+                }
+                // register metadata
+                serviceDiscovery.register(serviceInstance);
+            });
         }
-        publishMetadataToRemote(serviceInstance);
-
-        logger.info("Start registering instance address to registry.");
-        getServiceDiscoveries().forEach(serviceDiscovery ->
-        {
-            calInstanceRevision(serviceDiscovery, serviceInstance);
-            if (logger.isDebugEnabled()) {
-                logger.info("Start registering instance address to registry" + serviceDiscovery.getUrl() + ", instance " + serviceInstance);
-            }
-            // register metadata
-            serviceDiscovery.register(serviceInstance);
-        });
     }
 
     private void publishMetadataToRemote(ServiceInstance serviceInstance) {
 //        InMemoryWritableMetadataService localMetadataService = (InMemoryWritableMetadataService)WritableMetadataService.getDefaultExtension();
 //        localMetadataService.blockUntilUpdated();
+        if (logger.isInfoEnabled()) {
+            logger.info("Start publishing metadata to remote center, this only makes sense for applications enabled remote metadata center.");
+        }
         RemoteMetadataServiceImpl remoteMetadataService = MetadataUtils.getRemoteMetadataService();
         remoteMetadataService.publishMetadata(serviceInstance.getServiceName());
     }
 
-    private URL selectMetadataServiceExportedURL() {
-
-        URL selectedURL = null;
-
-        SortedSet<String> urlValues = metadataService.getExportedURLs();
-
-        for (String urlValue : urlValues) {
-            URL url = URL.valueOf(urlValue);
-            if (MetadataService.class.getName().equals(url.getServiceInterface())) {
-                continue;
-            }
-            if ("rest".equals(url.getProtocol())) { // REST first
-                selectedURL = url;
-                break;
-            } else {
-                selectedURL = url; // If not found, take any one
-            }
-        }
-
-        if (selectedURL == null && CollectionUtils.isNotEmpty(urlValues)) {
-            selectedURL = URL.valueOf(urlValues.iterator().next());
-        }
-
-        return selectedURL;
-    }
-
     private void unregisterServiceInstance() {
         if (serviceInstance != null) {
             getServiceDiscoveries().forEach(serviceDiscovery -> {
@@ -1226,18 +1189,10 @@ public class DubboBootstrap extends GenericEventListener {
         }
     }
 
-    private ServiceInstance createServiceInstance(String serviceName, String host, int port) {
-        this.serviceInstance = new DefaultServiceInstance(serviceName, host, port);
+    private ServiceInstance createServiceInstance(String serviceName) {
+        this.serviceInstance = new DefaultServiceInstance(serviceName);
         setMetadataStorageType(serviceInstance, getMetadataType());
-
-        ExtensionLoader<ServiceInstanceCustomizer> loader =
-                ExtensionLoader.getExtensionLoader(ServiceInstanceCustomizer.class);
-        // FIXME, sort customizer before apply
-        loader.getSupportedExtensionInstances().forEach(customizer -> {
-            // customizes
-            customizer.customize(this.serviceInstance);
-        });
-
+        ServiceInstanceMetadataUtils.customizeInstance(this.serviceInstance);
         return this.serviceInstance;
     }
 
@@ -1333,7 +1288,9 @@ public class DubboBootstrap extends GenericEventListener {
                     return applicationConfig;
                 });
 
-        application.refresh();
+        if (!application.isRefreshed()) {
+            application.refresh();
+        }
         return application;
     }
 
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ConfigurableMetadataServiceExporter.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ConfigurableMetadataServiceExporter.java
index fdb011f..edb2d0c 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ConfigurableMetadataServiceExporter.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ConfigurableMetadataServiceExporter.java
@@ -28,7 +28,6 @@ import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.MetadataServiceExporter;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 
-import java.util.ArrayList;
 import java.util.List;
 
 import static java.util.Collections.emptyList;
@@ -68,7 +67,7 @@ public class ConfigurableMetadataServiceExporter implements MetadataServiceExpor
 
             ServiceConfig<MetadataService> serviceConfig = new ServiceConfig<>();
             serviceConfig.setApplication(getApplicationConfig());
-            serviceConfig.setRegistries(getRegistries());
+            serviceConfig.setRegistry(new RegistryConfig("N/A"));
             serviceConfig.setProtocol(generateMetadataProtocol());
             serviceConfig.setInterface(MetadataService.class);
             serviceConfig.setRef(metadataService);
@@ -114,10 +113,6 @@ public class ConfigurableMetadataServiceExporter implements MetadataServiceExpor
         return ApplicationModel.getConfigManager().getApplication().get();
     }
 
-    private List<RegistryConfig> getRegistries() {
-        return new ArrayList<>(ApplicationModel.getConfigManager().getRegistries());
-    }
-
     private ProtocolConfig generateMetadataProtocol() {
         ProtocolConfig defaultProtocol = new ProtocolConfig();
         defaultProtocol.setName(DUBBO);
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstancePortCustomizer.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstanceHostPortCustomizer.java
similarity index 51%
rename from dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstancePortCustomizer.java
rename to dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstanceHostPortCustomizer.java
index db68898..3831337 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstancePortCustomizer.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/metadata/ServiceInstanceHostPortCustomizer.java
@@ -16,22 +16,22 @@
  */
 package org.apache.dubbo.config.metadata;
 
+import org.apache.dubbo.common.URL;
 import org.apache.dubbo.common.utils.CollectionUtils;
-import org.apache.dubbo.config.ProtocolConfig;
+import org.apache.dubbo.metadata.WritableMetadataService;
 import org.apache.dubbo.registry.client.DefaultServiceInstance;
 import org.apache.dubbo.registry.client.ServiceInstance;
 import org.apache.dubbo.registry.client.ServiceInstanceCustomizer;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 
-import java.util.Collection;
-import java.util.stream.Stream;
+import java.util.Set;
 
 /**
  * The {@link ServiceInstanceCustomizer} to customize the {@link ServiceInstance#getPort() port} of service instance.
  *
  * @since 2.7.5
  */
-public class ServiceInstancePortCustomizer implements ServiceInstanceCustomizer {
+public class ServiceInstanceHostPortCustomizer implements ServiceInstanceCustomizer {
 
     @Override
     public void customize(ServiceInstance serviceInstance) {
@@ -40,24 +40,31 @@ public class ServiceInstancePortCustomizer implements ServiceInstanceCustomizer
             return;
         }
 
-        Collection<ProtocolConfig> protocols = ApplicationModel.getConfigManager()
-                .getProtocols();
+        WritableMetadataService writableMetadataService = WritableMetadataService.getDefaultExtension();
 
-        if (CollectionUtils.isEmpty(protocols)) {
-            throw new IllegalStateException("We should have at least one protocol configured at this point.");
-        }
-
-        Stream<ProtocolConfig> protocolStream = protocols.stream();
-        ProtocolConfig protocolConfig = protocolStream
-                // use rest as service instance's default protocol.
-                .filter(protocol -> "rest".equals(protocol.getName()))
-                .findFirst()
-                .orElseGet(() -> protocolStream.findFirst().get());
-
-        if (serviceInstance instanceof DefaultServiceInstance) {
-            DefaultServiceInstance instance = (DefaultServiceInstance) serviceInstance;
-            if (protocolConfig.getPort() != null) {
-                instance.setPort(protocolConfig.getPort());
+        String host = null;
+        Integer port = null;
+        Set<URL> urls = writableMetadataService.getExportedServiceURLs();
+        if (CollectionUtils.isNotEmpty(urls)) {
+            String preferredProtocol = ApplicationModel.getApplicationConfig().getProtocol();
+            if (preferredProtocol != null) {
+                for (URL exportedURL : urls) {
+                    if (preferredProtocol.equals(exportedURL.getProtocol())) {
+                        host = exportedURL.getHost();
+                        port = exportedURL.getPort();
+                        break;
+                    }
+                }
+            } else {
+                URL url = urls.iterator().next();
+                host = url.getHost();
+                port = url.getPort();
+            }
+            if (serviceInstance instanceof DefaultServiceInstance) {
+                DefaultServiceInstance instance = (DefaultServiceInstance) serviceInstance;
+                instance.setHost(host);
+                instance.setPort(port);
+                instance.setId(host + ":" + port);
             }
         }
     }
diff --git a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/utils/ConfigValidationUtils.java b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/utils/ConfigValidationUtils.java
index 45cb9c6..ad42b51 100644
--- a/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/utils/ConfigValidationUtils.java
+++ b/dubbo-config/dubbo-config-api/src/main/java/org/apache/dubbo/config/utils/ConfigValidationUtils.java
@@ -87,9 +87,11 @@ import static org.apache.dubbo.common.constants.CommonConstants.SHUTDOWN_WAIT_SE
 import static org.apache.dubbo.common.constants.CommonConstants.THREADPOOL_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.USERNAME_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.VERSION_KEY;
+import static org.apache.dubbo.common.constants.RegistryConstants.DUBBO_PUBLISH_INSTANCE_DEFAULT_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.DUBBO_PUBLISH_INTERFACE_DEFAULT_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_PROTOCOL;
+import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_PUBLISH_INSTANCE_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_PUBLISH_INTERFACE_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_TYPE_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.SERVICE_REGISTRY_PROTOCOL;
@@ -212,18 +214,34 @@ public class ConfigValidationUtils {
     private static List<URL> genCompatibleRegistries(List<URL> registryList, boolean provider) {
         List<URL> result = new ArrayList<>(registryList.size());
         registryList.forEach(registryURL -> {
-            result.add(registryURL);
             if (provider) {
+                boolean publishInterface = registryURL.getParameter(REGISTRY_PUBLISH_INTERFACE_KEY, ConfigurationUtils.getDynamicGlobalConfiguration().getBoolean(DUBBO_PUBLISH_INTERFACE_DEFAULT_KEY, true));
                 // for registries enabled service discovery, automatically register interface compatible addresses.
-                if (SERVICE_REGISTRY_PROTOCOL.equals(registryURL.getProtocol())
-                        && registryURL.getParameter(REGISTRY_PUBLISH_INTERFACE_KEY, ConfigurationUtils.getDynamicGlobalConfiguration().getBoolean(DUBBO_PUBLISH_INTERFACE_DEFAULT_KEY, true))
-                        && registryNotExists(registryURL, registryList, REGISTRY_PROTOCOL)) {
-                    URL interfaceCompatibleRegistryURL = URLBuilder.from(registryURL)
-                            .setProtocol(REGISTRY_PROTOCOL)
-                            .removeParameter(REGISTRY_TYPE_KEY)
-                            .build();
-                    result.add(interfaceCompatibleRegistryURL);
+                if (SERVICE_REGISTRY_PROTOCOL.equals(registryURL.getProtocol())) {
+                    result.add(registryURL);
+                    if (publishInterface && registryNotExists(registryURL, registryList, REGISTRY_PROTOCOL)) {
+                        URL interfaceCompatibleRegistryURL = URLBuilder.from(registryURL)
+                                .setProtocol(REGISTRY_PROTOCOL)
+                                .removeParameter(REGISTRY_TYPE_KEY)
+                                .build();
+                        result.add(interfaceCompatibleRegistryURL);
+                    }
+                } else {
+                    boolean publishInstance = registryURL.getParameter(REGISTRY_PUBLISH_INSTANCE_KEY, ConfigurationUtils.getDynamicGlobalConfiguration().getBoolean(DUBBO_PUBLISH_INSTANCE_DEFAULT_KEY, true));
+                    if (registryNotExists(registryURL, registryList, SERVICE_REGISTRY_PROTOCOL)
+                            && publishInstance) {
+                        URL serviceDiscoveryRegistryURL = URLBuilder.from(registryURL)
+                                .setProtocol(SERVICE_REGISTRY_PROTOCOL)
+                                .removeParameter(REGISTRY_TYPE_KEY)
+                                .build();
+                        result.add(serviceDiscoveryRegistryURL);
+                    }
+                    if (publishInterface) {
+                        result.add(registryURL);
+                    }
                 }
+            } else {
+                result.add(registryURL);
             }
         });
         return result;
diff --git a/dubbo-config/dubbo-config-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.registry.client.ServiceInstanceCustomizer b/dubbo-config/dubbo-config-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.registry.client.ServiceInstanceCustomizer
index 3e9be3b..656f868 100644
--- a/dubbo-config/dubbo-config-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.registry.client.ServiceInstanceCustomizer
+++ b/dubbo-config/dubbo-config-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.registry.client.ServiceInstanceCustomizer
@@ -1 +1 @@
-port=org.apache.dubbo.config.metadata.ServiceInstancePortCustomizer
\ No newline at end of file
+port=org.apache.dubbo.config.metadata.ServiceInstanceHostPortCustomizer
\ No newline at end of file
diff --git a/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/compat/dubbo.xsd b/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/compat/dubbo.xsd
index ab229eb..cf20146 100644
--- a/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/compat/dubbo.xsd
+++ b/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/compat/dubbo.xsd
@@ -419,7 +419,16 @@
                 <xsd:documentation><![CDATA[ Register consumer instance or not, default false. ]]></xsd:documentation>
             </xsd:annotation>
         </xsd:attribute>
-
+        <xsd:attribute name="publish-interface" type="xsd:boolean">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ Publish interface level url to registry, default false. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
+        <xsd:attribute name="protocol" type="xsd:string">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ The preferred protocol to use, set protocol name. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
     </xsd:complexType>
 
     <xsd:complexType name="moduleType">
diff --git a/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/dubbo.xsd b/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/dubbo.xsd
index a79ebb4..57efa02 100644
--- a/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/dubbo.xsd
+++ b/dubbo-config/dubbo-config-spring/src/main/resources/META-INF/dubbo.xsd
@@ -415,6 +415,16 @@
                 <xsd:documentation><![CDATA[ Register consumer instance or not, default false. ]]></xsd:documentation>
             </xsd:annotation>
         </xsd:attribute>
+        <xsd:attribute name="publish-interface" type="xsd:boolean">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ Publish interface level url to registry, default true. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
+        <xsd:attribute name="protocol" type="xsd:string">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ The preferred protocol to use, set protocol name. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
     </xsd:complexType>
 
     <xsd:complexType name="moduleType">
@@ -623,6 +633,16 @@
                 <xsd:documentation><![CDATA[ weight of registry. ]]></xsd:documentation>
             </xsd:annotation>
         </xsd:attribute>
+        <xsd:attribute name="publish-interface" type="xsd:boolean">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ Publish interface level url to registry, default true. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
+        <xsd:attribute name="publish-instance" type="xsd:boolean">
+            <xsd:annotation>
+                <xsd:documentation><![CDATA[ Publish instance level url to registry, default true. ]]></xsd:documentation>
+            </xsd:annotation>
+        </xsd:attribute>
     </xsd:complexType>
 
     <xsd:complexType name="metadataReportType">
diff --git a/dubbo-demo/dubbo-demo-generic-call/pom.xml b/dubbo-demo/dubbo-demo-generic-call/pom.xml
new file mode 100644
index 0000000..0c53604
--- /dev/null
+++ b/dubbo-demo/dubbo-demo-generic-call/pom.xml
@@ -0,0 +1,71 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  ~ 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.
+  -->
+
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+    <parent>
+        <groupId>org.apache.dubbo</groupId>
+        <artifactId>dubbo-demo</artifactId>
+        <version>${revision}</version>
+        <relativePath>../pom.xml</relativePath>
+    </parent>
+    <modelVersion>4.0.0</modelVersion>
+
+    <artifactId>dubbo-demo-generic-call</artifactId>
+
+    <properties>
+        <skip_maven_deploy>true</skip_maven_deploy>
+    </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-config-api</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-registry-multicast</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-registry-zookeeper</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-configcenter-zookeeper</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-metadata-report-zookeeper</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-rpc-dubbo</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-remoting-netty4</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-serialization-hessian2</artifactId>
+        </dependency>
+    </dependencies>
+
+</project>
diff --git a/dubbo-demo/dubbo-demo-generic-call/src/main/java/org/apache/dubbo/demo/consumer/GenericApplication.java b/dubbo-demo/dubbo-demo-generic-call/src/main/java/org/apache/dubbo/demo/consumer/GenericApplication.java
new file mode 100644
index 0000000..65b2ff5
--- /dev/null
+++ b/dubbo-demo/dubbo-demo-generic-call/src/main/java/org/apache/dubbo/demo/consumer/GenericApplication.java
@@ -0,0 +1,85 @@
+/*
+ * 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.demo.consumer;
+
+import org.apache.dubbo.config.ApplicationConfig;
+import org.apache.dubbo.config.MetadataReportConfig;
+import org.apache.dubbo.config.ReferenceConfig;
+import org.apache.dubbo.config.RegistryConfig;
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
+import org.apache.dubbo.config.utils.ReferenceConfigCache;
+import org.apache.dubbo.rpc.service.GenericService;
+
+import java.util.HashMap;
+import java.util.Map;
+
+public class GenericApplication {
+    public static void main(String[] args) {
+        if (isClassic(args)) {
+//            runWithRefer();
+        } else {
+            runWithBootstrap();
+        }
+    }
+
+    private static boolean isClassic(String[] args) {
+        return args.length > 0 && "classic".equalsIgnoreCase(args[0]);
+    }
+
+    private static void runWithBootstrap() {
+        ReferenceConfig<GenericService> reference = new ReferenceConfig<>();
+        reference.setInterface("org.apache.dubbo.demo.DemoService");
+        reference.setGeneric("true");
+
+        ApplicationConfig applicationConfig = new ApplicationConfig("demo-consumer");
+        Map<String, String> parameters = new HashMap<>();
+        parameters.put("mapping-type", "metadata");
+        applicationConfig.setParameters(parameters);
+
+        MetadataReportConfig metadataReportConfig = new MetadataReportConfig();
+        metadataReportConfig.setAddress("zookeeper://127.0.0.1:2181");
+
+        DubboBootstrap bootstrap = DubboBootstrap.getInstance();
+        bootstrap.application(applicationConfig)
+                .registry(new RegistryConfig("zookeeper://127.0.0.1:2181"))
+                .reference(reference)
+                .start();
+
+        // generic invoke
+        GenericService genericService = (GenericService) ReferenceConfigCache.getCache().get(reference);
+        while (true) {
+            try {
+                Object genericInvokeResult = genericService.$invoke("sayHello", new String[]{String.class.getName()},
+                        new Object[]{"dubbo generic invoke"});
+                System.out.println(genericInvokeResult);
+                Thread.sleep(1000);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+//    private static void runWithRefer() {
+//        ReferenceConfig<DemoService> reference = new ReferenceConfig<>();
+//        reference.setApplication(new ApplicationConfig("dubbo-demo-api-consumer"));
+//        reference.setRegistry(new RegistryConfig("zookeeper://127.0.0.1:2181"));
+//        reference.setInterface(DemoService.class);
+//        DemoService service = reference.get();
+//        String message = service.sayHello("dubbo");
+//        System.out.println(message);
+//    }
+}
diff --git a/dubbo-demo/dubbo-demo-generic-call/src/main/resources/log4j.properties b/dubbo-demo/dubbo-demo-generic-call/src/main/resources/log4j.properties
new file mode 100644
index 0000000..15a0900
--- /dev/null
+++ b/dubbo-demo/dubbo-demo-generic-call/src/main/resources/log4j.properties
@@ -0,0 +1,7 @@
+###set log levels###
+log4j.rootLogger=info, stdout
+###output to the console###
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.Target=System.out
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=[%d{dd/MM/yy HH:mm:ss:SSS z}] %t %5p %c{2}: %m%n
\ No newline at end of file
diff --git a/dubbo-demo/dubbo-demo-interface/pom.xml b/dubbo-demo/dubbo-demo-interface/pom.xml
index 97aa7bf..b2b1cd7 100644
--- a/dubbo-demo/dubbo-demo-interface/pom.xml
+++ b/dubbo-demo/dubbo-demo-interface/pom.xml
@@ -29,4 +29,11 @@
     <properties>
         <skip_maven_deploy>true</skip_maven_deploy>
     </properties>
+
+    <dependencies>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-rpc-rest</artifactId>
+        </dependency>
+    </dependencies>
 </project>
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java b/dubbo-demo/dubbo-demo-interface/src/main/java/org/apache/dubbo/demo/RestDemoService.java
similarity index 57%
rename from dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java
rename to dubbo-demo/dubbo-demo-interface/src/main/java/org/apache/dubbo/demo/RestDemoService.java
index b8a7ac5..308b061 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/FilterChainBuilder.java
+++ b/dubbo-demo/dubbo-demo-interface/src/main/java/org/apache/dubbo/demo/RestDemoService.java
@@ -14,11 +14,32 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.dubbo.rpc;
+package org.apache.dubbo.demo;
 
-import org.apache.dubbo.common.extension.SPI;
+import javax.ws.rs.Consumes;
+import javax.ws.rs.GET;
+import javax.ws.rs.POST;
+import javax.ws.rs.Path;
+import javax.ws.rs.QueryParam;
+import javax.ws.rs.core.MediaType;
 
-@SPI("default")
-public interface FilterChainBuilder {
-    <T> Invoker<T> buildInvokerChain(final Invoker<T> invoker, String key, String group);
+
+@Path("/demoService")
+public interface RestDemoService {
+    @GET
+    @Path("/hello")
+    Integer hello(@QueryParam("a") Integer a, @QueryParam("b") Integer b);
+
+    @GET
+    @Path("/error")
+    String error();
+
+    @POST
+    @Path("/say")
+    @Consumes({MediaType.TEXT_PLAIN})
+    String sayHello(String name);
+
+    @GET
+    @Path("/getRemoteApplicationName")
+    String getRemoteApplicationName();
 }
diff --git a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/java/org/apache/dubbo/demo/consumer/Application.java b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/java/org/apache/dubbo/demo/consumer/Application.java
index 72b6583..afe8cb3 100644
--- a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/java/org/apache/dubbo/demo/consumer/Application.java
+++ b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/java/org/apache/dubbo/demo/consumer/Application.java
@@ -42,7 +42,7 @@ public class Application {
 
                     Thread.sleep(100);
                 } catch (Exception e) {
-                    e.printStackTrace();
+//                    e.printStackTrace();
                 }
             }
         }).start();
@@ -55,7 +55,7 @@ public class Application {
                 String greetings = greetingService.hello();
                 System.out.println("result: " + greetings);
             } catch (Exception e) {
-                e.printStackTrace();
+//                e.printStackTrace();
             }
 
             Thread.sleep(500);
diff --git a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/resources/spring/dubbo-consumer.xml b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/resources/spring/dubbo-consumer.xml
index f71fc5f..f0c6df1 100644
--- a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/resources/spring/dubbo-consumer.xml
+++ b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-consumer/src/main/resources/spring/dubbo-consumer.xml
@@ -27,7 +27,7 @@
 
     <!--    <dubbo:metadata-report address="zookeeper://127.0.0.1:2181"/>-->
 
-    <dubbo:registry address="zookeeper://127.0.0.1:2181?registry-type=service"/>
+    <dubbo:registry address="zookeeper://127.0.0.1:2181"/>
 
     <dubbo:reference id="demoService" check="false"
                      interface="org.apache.dubbo.demo.DemoService"/>
diff --git a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/pom.xml b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/pom.xml
index 20e3b52..c8cf138 100644
--- a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/pom.xml
+++ b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/pom.xml
@@ -72,6 +72,10 @@
         </dependency>
         <dependency>
             <groupId>org.apache.dubbo</groupId>
+            <artifactId>dubbo-rpc-rest</artifactId>
+        </dependency>
+        <dependency>
+            <groupId>org.apache.dubbo</groupId>
             <artifactId>dubbo-config-spring</artifactId>
         </dependency>
         <dependency>
diff --git a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/java/org/apache/dubbo/demo/provider/RestDemoServiceImpl.java b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/java/org/apache/dubbo/demo/provider/RestDemoServiceImpl.java
new file mode 100644
index 0000000..8936e23
--- /dev/null
+++ b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/java/org/apache/dubbo/demo/provider/RestDemoServiceImpl.java
@@ -0,0 +1,58 @@
+/*
+ * 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.demo.provider;
+
+
+import org.apache.dubbo.demo.RestDemoService;
+import org.apache.dubbo.rpc.RpcContext;
+
+import java.util.Map;
+
+public class RestDemoServiceImpl implements RestDemoService {
+    private static Map<String, Object> context;
+    private boolean called;
+
+    public String sayHello(String name) {
+        called = true;
+        return "Hello, " + name;
+    }
+
+
+    public boolean isCalled() {
+        return called;
+    }
+
+    @Override
+    public Integer hello(Integer a, Integer b) {
+        context = RpcContext.getContext().getObjectAttachments();
+        return a + b;
+    }
+
+    @Override
+    public String error() {
+        throw new RuntimeException();
+    }
+
+    public static Map<String, Object> getAttachments() {
+        return context;
+    }
+
+    @Override
+    public String getRemoteApplicationName() {
+        return RpcContext.getContext().getRemoteApplicationName();
+    }
+}
diff --git a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/resources/spring/dubbo-provider.xml b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/resources/spring/dubbo-provider.xml
index 856ff73..386be3c 100644
--- a/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/resources/spring/dubbo-provider.xml
+++ b/dubbo-demo/dubbo-demo-xml/dubbo-demo-xml-provider/src/main/resources/spring/dubbo-provider.xml
@@ -30,12 +30,16 @@
     <dubbo:registry id="registry1" address="zookeeper://127.0.0.1:2181?registry-type=service"/>
 
     <dubbo:protocol name="dubbo" port="-1"/>
+    <dubbo:protocol name="rest" port="-1"/>
 
     <bean id="demoService" class="org.apache.dubbo.demo.provider.DemoServiceImpl"/>
     <bean id="greetingService" class="org.apache.dubbo.demo.provider.GreetingServiceImpl"/>
+    <bean id="restDemoService" class="org.apache.dubbo.demo.provider.RestDemoServiceImpl"/>
 
-    <dubbo:service interface="org.apache.dubbo.demo.DemoService" timeout="3000" ref="demoService" registry="registry1"/>
-    <dubbo:service version="1.0.0" group="greeting" timeout="5000" interface="org.apache.dubbo.demo.GreetingService"
-                   ref="greetingService"/>
+    <dubbo:service delay="5000" interface="org.apache.dubbo.demo.DemoService" timeout="3000" ref="demoService" registry="registry1" protocol="dubbo"/>
+    <dubbo:service delay="5000" version="1.0.0" group="greeting" timeout="5000" interface="org.apache.dubbo.demo.GreetingService"
+                   ref="greetingService" protocol="dubbo"/>
+    <dubbo:service delay="5000" version="1.0.0" timeout="5000" interface="org.apache.dubbo.demo.RestDemoService"
+                   ref="restDemoService" protocol="rest"/>
 
 </beans>
diff --git a/dubbo-demo/pom.xml b/dubbo-demo/pom.xml
index e636a23..573615a 100644
--- a/dubbo-demo/pom.xml
+++ b/dubbo-demo/pom.xml
@@ -35,6 +35,7 @@
         <module>dubbo-demo-xml</module>
         <module>dubbo-demo-annotation</module>
         <module>dubbo-demo-api</module>
+        <module>dubbo-demo-generic-call</module>
     </modules>
 
     <dependencyManagement>
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataConstants.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataConstants.java
index 6bf38c9..ac88d75 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataConstants.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataConstants.java
@@ -24,7 +24,7 @@ public class MetadataConstants {
     public static final String SERVICE_META_DATA_STORE_TAG = ".smd";
     public static final String CONSUMER_META_DATA_STORE_TAG = ".cmd";
     public static final String METADATA_PUBLISH_DELAY_KEY = "dubbo.application.metadata.delay";
-    public static final int DEFAULT_METADATA_PUBLISH_DELAY = 5000;
+    public static final int DEFAULT_METADATA_PUBLISH_DELAY = 10000;
     public static final String METADATA_PROXY_TIMEOUT_KEY = "dubbo.application.metadata.delay";
     public static final int DEFAULT_METADATA_TIMEOUT_VALUE = 5000;
 }
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataInfo.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataInfo.java
index 0ad553d..55c6ed9 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataInfo.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataInfo.java
@@ -97,10 +97,14 @@ public class MetadataInfo implements Serializable {
 
         StringBuilder sb = new StringBuilder();
         sb.append(app);
-        for (Map.Entry<String, ServiceInfo> entry : new TreeMap<>(services).entrySet()) {
-            sb.append(entry.getValue().toDescString());
+        if (CollectionUtils.isEmptyMap(services)) {
+            this.revision = RevisionResolver.getEmptyRevision(app);
+        } else {
+            for (Map.Entry<String, ServiceInfo> entry : new TreeMap<>(services).entrySet()) {
+                sb.append(entry.getValue().toDescString());
+            }
+            this.revision = RevisionResolver.calRevision(sb.toString());
         }
-        this.revision = RevisionResolver.calRevision(sb.toString());
         return revision;
     }
 
@@ -203,7 +207,7 @@ public class MetadataInfo implements Serializable {
         // service + group + version + protocol
         private transient String matchKey;
 
-        private URL url;
+        private transient URL url;
 
         public ServiceInfo() {
         }
@@ -305,6 +309,14 @@ public class MetadataInfo implements Serializable {
             this.path = path;
         }
 
+        public String getProtocol() {
+            return protocol;
+        }
+
+        public void setProtocol(String protocol) {
+            this.protocol = protocol;
+        }
+
         public Map<String, String> getParams() {
             if (params == null) {
                 return Collections.emptyMap();
@@ -357,9 +369,6 @@ public class MetadataInfo implements Serializable {
                 if (keyMap != null) {
                     value = keyMap.get(key);
                 }
-                if (StringUtils.isEmpty(value)) {
-                    value = getParameter(key);
-                }
             }
             return value;
         }
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataService.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataService.java
index 761cd88..24dc82d 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataService.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/MetadataService.java
@@ -18,6 +18,7 @@ package org.apache.dubbo.metadata;
 
 import org.apache.dubbo.common.URL;
 
+import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -162,6 +163,10 @@ public interface MetadataService {
      */
     SortedSet<String> getExportedURLs(String serviceInterface, String group, String version, String protocol);
 
+    default Set<URL> getExportedServiceURLs() {
+        return Collections.emptySet();
+    }
+
     /**
      * Interface definition.
      *
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/RevisionResolver.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/RevisionResolver.java
index f8ebd3b..8e7ffe4 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/RevisionResolver.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/RevisionResolver.java
@@ -26,6 +26,7 @@ import static java.nio.charset.StandardCharsets.UTF_8;
 
 public class RevisionResolver {
     private static final Logger logger = LoggerFactory.getLogger(RevisionResolver.class);
+    private static final String EMPTY_REVISION = "0";
     private static char hexDigits[] = {
             '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'
     };
@@ -55,5 +56,8 @@ public class RevisionResolver {
         return new String(str);
     }
 
+    public static String getEmptyRevision(String app) {
+        return EMPTY_REVISION;
+    }
 
 }
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 074080f..5282dea 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
@@ -22,6 +22,8 @@ import org.apache.dubbo.common.extension.SPI;
 import java.util.Set;
 
 import static org.apache.dubbo.common.constants.CommonConstants.CONFIG_MAPPING_TYPE;
+import static org.apache.dubbo.common.constants.CommonConstants.DUBBO;
+import static org.apache.dubbo.common.constants.CommonConstants.PROTOCOL_KEY;
 import static org.apache.dubbo.common.extension.ExtensionLoader.getExtensionLoader;
 import static org.apache.dubbo.common.utils.StringUtils.SLASH;
 import static org.apache.dubbo.metadata.DynamicConfigurationServiceNameMapping.DEFAULT_MAPPING_GROUP;
@@ -46,6 +48,14 @@ public interface ServiceNameMapping {
      */
     Set<String> getAndListen(URL url, MappingListener mappingListener);
 
+    default Set<String> get(URL url) {
+        return getAndListen(url, null);
+    }
+
+    static String toStringKeys(Set<String> serviceNames) {
+        return serviceNames.toString();
+    }
+
     /**
      * Get the default extension of {@link ServiceNameMapping}
      *
@@ -60,6 +70,10 @@ public interface ServiceNameMapping {
         return getExtensionLoader(ServiceNameMapping.class).getExtension(name == null ? CONFIG_MAPPING_TYPE : name);
     }
 
+    static String buildMappingKey(URL url) {
+        return buildGroup(url.getServiceInterface(), url.getGroup(), url.getVersion(), url.getParameter(PROTOCOL_KEY, DUBBO));
+    }
+
     static String buildGroup(String serviceInterface, String group, String version, String protocol) {
         //        the issue : https://github.com/apache/dubbo/issues/4671
         //        StringBuilder groupBuilder = new StringBuilder(serviceInterface)
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/WritableMetadataService.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/WritableMetadataService.java
index b870fff..6a044cf 100644
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/WritableMetadataService.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/WritableMetadataService.java
@@ -21,6 +21,8 @@ import org.apache.dubbo.common.extension.ExtensionLoader;
 import org.apache.dubbo.common.extension.SPI;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 
+import java.util.Set;
+
 import static org.apache.dubbo.common.extension.ExtensionLoader.getExtensionLoader;
 
 /**
@@ -75,6 +77,19 @@ public interface WritableMetadataService extends MetadataService {
 
     void publishServiceDefinition(URL providerUrl);
 
+    default void setMetadataServiceURL(URL url) {
+
+    }
+
+    default URL getMetadataServiceURL() {
+        return null;
+    }
+
+    void putCachedMapping(String serviceKey, Set<String> apps);
+
+    Set<String> getCachedMapping(URL consumerURL);
+
+    Set<String> removeCachedMapping(String serviceKey);
 
     /**
      * Get {@link ExtensionLoader#getDefaultExtension() the defautl extension} of {@link WritableMetadataService}
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 20a7742..f06cca3 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
@@ -61,7 +61,7 @@ public class ZookeeperMetadataReport extends AbstractMetadataReport {
 
     private Gson gson = new Gson();
 
-    private Map<String, ChildListener> listenerMap = new ConcurrentHashMap<>();
+    private Map<String, MappingChildListener> listenerMap = new ConcurrentHashMap<>();
 
     public ZookeeperMetadataReport(URL url, ZookeeperTransporter zookeeperTransporter) {
         super(url);
@@ -177,16 +177,32 @@ public class ZookeeperMetadataReport extends AbstractMetadataReport {
     }
 
     private void addServiceMappingListener(String path, String serviceKey, MappingListener listener) {
-        ChildListener zkListener = new ChildListener() {
-            @Override
-            public void childChanged(String path, List<String> children) {
-                MappingChangedEvent event = new MappingChangedEvent();
-                event.setServiceKey(serviceKey);
-                event.setApps(null != children ? new HashSet<>(children) : null);
-                listener.onEvent(event);
-            }
-        };
-        zkClient.addChildListener(path, zkListener);
-        listenerMap.put(path, zkListener);
+        MappingChildListener mappingChildListener = listenerMap.computeIfAbsent(path, _k -> new MappingChildListener(serviceKey, path));
+        mappingChildListener.addListener(listener);
+        zkClient.addChildListener(path, mappingChildListener);
+    }
+
+    private static class MappingChildListener implements ChildListener {
+        private String serviceKey;
+        private String path;
+        private Set<MappingListener> listeners;
+
+        public MappingChildListener(String serviceKey, String path) {
+            this.serviceKey = serviceKey;
+            this.path = path;
+            this.listeners = new HashSet<>();
+        }
+
+        public void addListener(MappingListener listener) {
+            this.listeners.add(listener);
+        }
+
+        @Override
+        public void childChanged(String path, List<String> children) {
+            MappingChangedEvent event = new MappingChangedEvent();
+            event.setServiceKey(serviceKey);
+            event.setApps(null != children ? new HashSet<>(children) : null);
+            listeners.forEach(mappingListener -> mappingListener.onEvent(event));
+        }
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOffline.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOffline.java
index 4323c04..4aeaa92 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOffline.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOffline.java
@@ -21,9 +21,11 @@ import org.apache.dubbo.common.logger.Logger;
 import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.qos.command.BaseCommand;
 import org.apache.dubbo.qos.command.CommandContext;
+import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.registry.RegistryFactory;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 import org.apache.dubbo.rpc.model.ProviderModel;
+import org.apache.dubbo.rpc.model.ServiceMetadata;
 import org.apache.dubbo.rpc.model.ServiceRepository;
 
 import java.util.Collection;
@@ -60,7 +62,8 @@ public class BaseOffline implements BaseCommand {
 
         Collection<ProviderModel> providerModelList = serviceRepository.getExportedServices();
         for (ProviderModel providerModel : providerModelList) {
-            if (providerModel.getServiceMetadata().getServiceKey().matches(servicePattern)) {
+            ServiceMetadata metadata = providerModel.getServiceMetadata();
+            if (metadata.getServiceKey().matches(servicePattern) || metadata.getDisplayServiceKey().matches(servicePattern)) {
                 hasService = true;
                 List<ProviderModel.RegisterStatedURL> statedUrls = providerModel.getStatedUrl();
                 for (ProviderModel.RegisterStatedURL statedURL : statedUrls) {
@@ -74,5 +77,9 @@ public class BaseOffline implements BaseCommand {
         return hasService;
     }
 
-    protected void doUnexport(ProviderModel.RegisterStatedURL statedURL) {}
+    protected void doUnexport(ProviderModel.RegisterStatedURL statedURL) {
+        Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
+        registry.unregister(statedURL.getProviderUrl());
+        statedURL.setRegistered(false);
+    }
 }
\ No newline at end of file
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOnline.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOnline.java
index bd1f276..cf01ff1 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOnline.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/BaseOnline.java
@@ -22,9 +22,11 @@ import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.common.utils.ArrayUtils;
 import org.apache.dubbo.qos.command.BaseCommand;
 import org.apache.dubbo.qos.command.CommandContext;
+import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.registry.RegistryFactory;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 import org.apache.dubbo.rpc.model.ProviderModel;
+import org.apache.dubbo.rpc.model.ServiceMetadata;
 import org.apache.dubbo.rpc.model.ServiceRepository;
 
 import java.util.Collection;
@@ -56,7 +58,8 @@ public class BaseOnline implements BaseCommand {
 
         Collection<ProviderModel> providerModelList = serviceRepository.getExportedServices();
         for (ProviderModel providerModel : providerModelList) {
-            if (providerModel.getServiceMetadata().getServiceKey().matches(servicePattern)) {
+            ServiceMetadata metadata = providerModel.getServiceMetadata();
+            if (metadata.getServiceKey().matches(servicePattern) || metadata.getDisplayServiceKey().matches(servicePattern)) {
                 hasService = true;
                 List<ProviderModel.RegisterStatedURL> statedUrls = providerModel.getStatedUrl();
                 for (ProviderModel.RegisterStatedURL statedURL : statedUrls) {
@@ -75,6 +78,8 @@ public class BaseOnline implements BaseCommand {
     }
 
     protected void doExport(ProviderModel.RegisterStatedURL statedURL) {
-
+        Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
+        registry.register(statedURL.getProviderUrl());
+        statedURL.setRegistered(true);
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Offline.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Offline.java
index 6b95a17..5d1031c 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Offline.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Offline.java
@@ -16,25 +16,16 @@
  */
 package org.apache.dubbo.qos.command.impl;
 
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.qos.command.annotation.Cmd;
+import org.apache.dubbo.rpc.model.ProviderModel;
 
 @Cmd(name = "offline", summary = "offline dubbo", example = {
         "offline dubbo",
         "offline xx.xx.xxx.service"
 })
 public class Offline extends BaseOffline {
-    private Logger logger = LoggerFactory.getLogger(Offline.class);
-
-    private static OfflineInterface offlineInterface = new OfflineInterface();
-    private static OfflineApp offlineApp = new OfflineApp();
-
     @Override
-    protected boolean doExecute(String servicePattern) {
-        boolean r1 = offlineInterface.offline(servicePattern);
-        boolean r2 = offlineApp.offline(servicePattern);
-        return r1 && r2;
+    protected void doUnexport(ProviderModel.RegisterStatedURL statedURL) {
+        super.doUnexport(statedURL);
     }
-
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineApp.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineApp.java
index 5920e03..a3ef538 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineApp.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineApp.java
@@ -16,9 +16,10 @@
  */
 package org.apache.dubbo.qos.command.impl;
 
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.common.utils.UrlUtils;
 import org.apache.dubbo.qos.command.annotation.Cmd;
-import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.rpc.model.ProviderModel;
 
 @Cmd(name = "offlineApp", summary = "offline app addresses", example = {
@@ -26,12 +27,12 @@ import org.apache.dubbo.rpc.model.ProviderModel;
         "offlineApp xx.xx.xxx.service"
 })
 public class OfflineApp extends BaseOffline {
+    private static final Logger logger = LoggerFactory.getLogger(OfflineApp.class);
+
     @Override
     protected void doUnexport(ProviderModel.RegisterStatedURL statedURL) {
         if (UrlUtils.isServiceDiscoveryRegistryType(statedURL.getRegistryUrl())) {
-            Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
-            registry.unregister(statedURL.getProviderUrl());
-            statedURL.setRegistered(false);
+            super.doUnexport(statedURL);
         }
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineInterface.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineInterface.java
index d4f959d..57aa72a 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineInterface.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OfflineInterface.java
@@ -18,7 +18,6 @@ package org.apache.dubbo.qos.command.impl;
 
 import org.apache.dubbo.common.utils.UrlUtils;
 import org.apache.dubbo.qos.command.annotation.Cmd;
-import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.rpc.model.ProviderModel;
 
 @Cmd(name = "offlineInterface", summary = "offline dubbo", example = {
@@ -28,9 +27,7 @@ import org.apache.dubbo.rpc.model.ProviderModel;
 public class OfflineInterface extends BaseOffline {
     protected void doUnexport(ProviderModel.RegisterStatedURL statedURL) {
         if (!UrlUtils.isServiceDiscoveryRegistryType(statedURL.getRegistryUrl())) {
-            Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
-            registry.unregister(statedURL.getProviderUrl());
-            statedURL.setRegistered(false);
+            super.doUnexport(statedURL);
         }
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Online.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Online.java
index e5f39d8..d2f3d03 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Online.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/Online.java
@@ -16,23 +16,16 @@
  */
 package org.apache.dubbo.qos.command.impl;
 
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.qos.command.annotation.Cmd;
+import org.apache.dubbo.rpc.model.ProviderModel;
 
 @Cmd(name = "onlineAPp", summary = "offline app addresses", example = {
         "offlineApp dubbo",
         "offlineApp xx.xx.xxx.service"
 })
 public class Online extends BaseOnline {
-    private static final Logger logger = LoggerFactory.getLogger(Online.class);
-    private static final OnlineInterface onlineInteface = new OnlineInterface();
-    private static final OnlineApp onlineApp = new OnlineApp();
-
     @Override
-    protected boolean doExecute(String servicePattern) {
-        boolean r1 = onlineInteface.online(servicePattern);
-        boolean r2 = onlineApp.online(servicePattern);
-        return r1 && r2;
+    protected void doExport(ProviderModel.RegisterStatedURL statedURL) {
+        super.doExport(statedURL);
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineApp.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineApp.java
index 799b72e..5ba3b9a 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineApp.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineApp.java
@@ -17,17 +17,18 @@
 package org.apache.dubbo.qos.command.impl;
 
 
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
 import org.apache.dubbo.common.utils.UrlUtils;
-import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.rpc.model.ProviderModel;
 
 public class OnlineApp extends BaseOnline {
+    private static final Logger logger = LoggerFactory.getLogger(OnlineApp.class);
+
     @Override
     protected void doExport(ProviderModel.RegisterStatedURL statedURL) {
-        if (!UrlUtils.isServiceDiscoveryRegistryType(statedURL.getRegistryUrl())) {
-            Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
-            registry.register(statedURL.getProviderUrl());
-            statedURL.setRegistered(true);
+        if (UrlUtils.isServiceDiscoveryRegistryType(statedURL.getRegistryUrl())) {
+           super.doExport(statedURL);
         }
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineInterface.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineInterface.java
index 91e8eb6..ac0e5cd 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineInterface.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/OnlineInterface.java
@@ -17,16 +17,13 @@
 package org.apache.dubbo.qos.command.impl;
 
 import org.apache.dubbo.common.utils.UrlUtils;
-import org.apache.dubbo.registry.Registry;
 import org.apache.dubbo.rpc.model.ProviderModel;
 
 public class OnlineInterface extends BaseOnline {
     @Override
     protected void doExport(ProviderModel.RegisterStatedURL statedURL) {
         if (!UrlUtils.isServiceDiscoveryRegistryType(statedURL.getRegistryUrl())) {
-            Registry registry = registryFactory.getRegistry(statedURL.getRegistryUrl());
-            registry.register(statedURL.getProviderUrl());
-            statedURL.setRegistered(true);
+            super.doExport(statedURL);
         }
     }
 }
diff --git a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/PublishMetadata.java b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/PublishMetadata.java
index 990854c..34ef7d0 100644
--- a/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/PublishMetadata.java
+++ b/dubbo-plugin/dubbo-qos/src/main/java/org/apache/dubbo/qos/command/impl/PublishMetadata.java
@@ -43,7 +43,7 @@ public class PublishMetadata implements BaseCommand {
         logger.info("received publishMetadata command.");
 
         if (ArrayUtils.isEmpty(args)) {
-            ServiceInstanceMetadataUtils.refreshMetadataAndInstance();
+            ServiceInstanceMetadataUtils.refreshMetadataAndInstance(null);
             return "publish metadata succeeded.";
         }
 
@@ -51,7 +51,7 @@ public class PublishMetadata implements BaseCommand {
             int delay = Integer.parseInt(args[0]);
             if (future == null || future.isDone() || future.isCancelled()) {
                 future = executorRepository.nextScheduledExecutor()
-                        .scheduleWithFixedDelay(ServiceInstanceMetadataUtils::refreshMetadataAndInstance, 0, delay, TimeUnit.MILLISECONDS);
+                        .scheduleWithFixedDelay(() -> ServiceInstanceMetadataUtils.refreshMetadataAndInstance(null), 0, delay, TimeUnit.MILLISECONDS);
             }
         } catch (NumberFormatException e) {
             logger.error("Wrong delay param", e);
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/DefaultServiceInstance.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/DefaultServiceInstance.java
index baaaa66..241acfe 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/DefaultServiceInstance.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/DefaultServiceInstance.java
@@ -18,13 +18,17 @@ package org.apache.dubbo.registry.client;
 
 import org.apache.dubbo.metadata.MetadataInfo;
 
+import com.alibaba.fastjson.JSON;
+
 import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.SortedMap;
 import java.util.TreeMap;
 
 import static org.apache.dubbo.common.constants.CommonConstants.REVISION_KEY;
+import static org.apache.dubbo.registry.client.metadata.ServiceInstanceMetadataUtils.ENDPOINTS;
 
 /**
  * The default implementation of {@link ServiceInstance}.
@@ -53,10 +57,24 @@ public class DefaultServiceInstance implements ServiceInstance {
     private transient MetadataInfo serviceMetadata;
     // used at runtime
     private transient Map<String, String> extendParams = new HashMap<>();
+    private transient List<Endpoint> endpoints;
 
     public DefaultServiceInstance() {
     }
 
+    public DefaultServiceInstance(DefaultServiceInstance other) {
+        this.id = other.id;
+        this.serviceName = other.serviceName;
+        this.host = other.host;
+        this.port = other.port;
+        this.enabled = other.enabled;
+        this.healthy = other.healthy;
+        this.metadata = other.metadata;
+        this.serviceMetadata = other.serviceMetadata;
+        this.extendParams = other.extendParams;
+        this.endpoints = other.endpoints;
+    }
+
     public DefaultServiceInstance(String id, String serviceName, String host, Integer port) {
         if (port != null && port.intValue() < 1) {
             throw new IllegalArgumentException("The port must be greater than zero!");
@@ -73,6 +91,10 @@ public class DefaultServiceInstance implements ServiceInstance {
         this(host + ":" + port, serviceName, host, port);
     }
 
+    public DefaultServiceInstance(String serviceName) {
+        this.serviceName = serviceName;
+    }
+
     public void setId(String id) {
         this.id = id;
     }
@@ -154,6 +176,20 @@ public class DefaultServiceInstance implements ServiceInstance {
         return extendParams;
     }
 
+    public List<Endpoint> getEndpoints() {
+        if (endpoints != null) {
+            return endpoints;
+        }
+        return JSON.parseArray(metadata.get(ENDPOINTS), Endpoint.class);
+    }
+
+    public DefaultServiceInstance copy(Endpoint endpoint) {
+        DefaultServiceInstance copyOfInstance = new DefaultServiceInstance(this);
+        copyOfInstance.setPort(endpoint.getPort());
+//        copyOfInstance.setId(copyOfInstance.getAddress());
+        return copyOfInstance;
+    }
+
     @Override
     public Map<String, String> getAllParams() {
         Map<String, String> allParams = new HashMap<>((int) ((metadata.size() + extendParams.size()) / 0.75f + 1));
@@ -221,4 +257,30 @@ public class DefaultServiceInstance implements ServiceInstance {
                 ", metadata=" + metadata +
                 '}';
     }
+
+    public static class Endpoint {
+        Integer port;
+        String protocol;
+
+        public Endpoint(Integer port, String protocol) {
+            this.port = port;
+            this.protocol = protocol;
+        }
+
+        public Integer getPort() {
+            return port;
+        }
+
+        public void setPort(Integer port) {
+            this.port = port;
+        }
+
+        public String getProtocol() {
+            return protocol;
+        }
+
+        public void setProtocol(String protocol) {
+            this.protocol = protocol;
+        }
+    }
 }
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 4cd058d..8ca6cd3 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,9 +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;
 import org.apache.dubbo.metadata.WritableMetadataService;
 import org.apache.dubbo.registry.NotifyListener;
@@ -38,37 +35,27 @@ import org.apache.dubbo.registry.support.FailbackRegistry;
 import java.util.ArrayList;
 import java.util.Collections;
 import java.util.HashMap;
-import java.util.LinkedHashMap;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 import java.util.Set;
-import java.util.TreeSet;
 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;
 import static org.apache.dubbo.common.constants.CommonConstants.GROUP_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.INTERFACE_KEY;
-import static org.apache.dubbo.common.constants.CommonConstants.MAPPING_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;
 import static org.apache.dubbo.rpc.Constants.ID_KEY;
 
@@ -101,32 +88,18 @@ public class ServiceDiscoveryRegistry implements Registry {
 
     private final ServiceDiscovery serviceDiscovery;
 
-    private final Set<String> subscribedServices;
-
-    private final ServiceNameMapping serviceNameMapping;
-
     private final WritableMetadataService writableMetadataService;
 
     private final Set<String> registeredListeners = new LinkedHashSet<>();
 
     /* apps - listener */
     private final Map<String, ServiceInstancesChangedListener> serviceListeners = new HashMap<>();
-    private final Map<String, String> serviceToAppsMapping = new HashMap<>();
 
     private URL registryURL;
 
-    /**
-     * A cache for all URLs of services that the subscribed services exported
-     * The key is the service name
-     * The value is a nested {@link Map} whose key is the revision and value is all URLs of services
-     */
-    private final Map<String, Map<String, List<URL>>> serviceRevisionExportedURLsCache = new LinkedHashMap<>();
-
     public ServiceDiscoveryRegistry(URL registryURL) {
         this.registryURL = registryURL;
         this.serviceDiscovery = createServiceDiscovery(registryURL);
-        this.subscribedServices = parseServices(registryURL.getParameter(SUBSCRIBED_SERVICE_NAMES_KEY));
-        this.serviceNameMapping = ServiceNameMapping.getExtension(registryURL.getParameter(MAPPING_KEY));
         this.writableMetadataService = WritableMetadataService.getDefaultExtension();
     }
 
@@ -205,10 +178,7 @@ public class ServiceDiscoveryRegistry implements Registry {
     }
 
     public void doRegister(URL url) {
-        String registryCluster = serviceDiscovery.getUrl().getParameter(ID_KEY);
-        if (registryCluster != null && url.getParameter(REGISTRY_CLUSTER_KEY) == null) {
-            url = url.addParameter(REGISTRY_CLUSTER_KEY, registryCluster);
-        }
+        url = addRegistryClusterKey(url);
         if (writableMetadataService.exportURL(url)) {
             if (logger.isInfoEnabled()) {
                 logger.info(format("The URL[%s] registered successfully.", url.toString()));
@@ -229,10 +199,7 @@ public class ServiceDiscoveryRegistry implements Registry {
     }
 
     public void doUnregister(URL url) {
-        String registryCluster = serviceDiscovery.getUrl().getParameter(ID_KEY);
-        if (registryCluster != null && url.getParameter(REGISTRY_CLUSTER_KEY) == null) {
-            url = url.addParameter(REGISTRY_CLUSTER_KEY, registryCluster);
-        }
+        url = addRegistryClusterKey(url);
         if (writableMetadataService.unexportURL(url)) {
             if (logger.isInfoEnabled()) {
                 logger.info(format("The URL[%s] deregistered successfully.", url.toString()));
@@ -249,10 +216,7 @@ public class ServiceDiscoveryRegistry implements Registry {
         if (!shouldSubscribe(url)) { // Should Not Subscribe
             return;
         }
-        String registryCluster = serviceDiscovery.getUrl().getParameter(ID_KEY);
-        if (registryCluster != null && url.getParameter(REGISTRY_CLUSTER_KEY) == null) {
-            url = url.addParameter(REGISTRY_CLUSTER_KEY, registryCluster);
-        }
+        url = addRegistryClusterKey(url);
         doSubscribe(url, listener);
     }
 
@@ -260,7 +224,7 @@ public class ServiceDiscoveryRegistry implements Registry {
         writableMetadataService.subscribeURL(url);
 
         boolean check = url.getParameter(CHECK_KEY, false);
-        Set<String> serviceNames = getServices(url, listener);
+        Set<String> serviceNames = writableMetadataService.getCachedMapping(url);
 
         if (CollectionUtils.isEmpty(serviceNames)) {
             if (check) {
@@ -277,19 +241,24 @@ public class ServiceDiscoveryRegistry implements Registry {
         if (!shouldSubscribe(url)) { // Should Not Subscribe
             return;
         }
+        url = addRegistryClusterKey(url);
+        doUnsubscribe(url, listener);
+    }
+
+    private URL addRegistryClusterKey(URL url) {
         String registryCluster = serviceDiscovery.getUrl().getParameter(ID_KEY);
         if (registryCluster != null && url.getParameter(REGISTRY_CLUSTER_KEY) == null) {
             url = url.addParameter(REGISTRY_CLUSTER_KEY, registryCluster);
         }
-        doUnsubscribe(url, listener);
+        return url;
     }
 
     public void doUnsubscribe(URL url, NotifyListener listener) {
         writableMetadataService.unsubscribeURL(url);
         String protocolServiceKey = url.getServiceKey() + GROUP_CHAR_SEPARATOR + url.getParameter(PROTOCOL_KEY, DUBBO);
-        String serviceNamesKey = serviceToAppsMapping.remove(protocolServiceKey);
-        if (serviceNamesKey != null) {
-            ServiceInstancesChangedListener instancesChangedListener = serviceListeners.get(serviceNamesKey);
+        Set<String> serviceNames = writableMetadataService.removeCachedMapping(ServiceNameMapping.buildMappingKey(url));
+        if (CollectionUtils.isNotEmpty(serviceNames)) {
+            ServiceInstancesChangedListener instancesChangedListener = serviceListeners.get(toStringKeys(serviceNames));
             instancesChangedListener.removeListener(protocolServiceKey);
         }
     }
@@ -319,9 +288,8 @@ public class ServiceDiscoveryRegistry implements Registry {
     }
 
     protected void subscribeURLs(URL url, NotifyListener listener, Set<String> serviceNames) {
-        String serviceNamesKey = serviceNames.toString();
+        String serviceNamesKey = toStringKeys(serviceNames);
         String protocolServiceKey = url.getServiceKey() + GROUP_CHAR_SEPARATOR + url.getParameter(PROTOCOL_KEY, DUBBO);
-        serviceToAppsMapping.put(protocolServiceKey, serviceNamesKey);
 
         // register ServiceInstancesChangedListener
         ServiceInstancesChangedListener serviceListener = serviceListeners.computeIfAbsent(serviceNamesKey, k -> {
@@ -338,10 +306,8 @@ public class ServiceDiscoveryRegistry implements Registry {
 
         serviceListener.setUrl(url);
         listener.addServiceListener(serviceListener);
-        serviceListener.addListener(protocolServiceKey, listener);
+        serviceListener.addListenerAndNotify(protocolServiceKey, listener);
         registerServiceInstancesChangedListener(url, serviceListener);
-
-        listener.notify(serviceListener.getUrls(protocolServiceKey));
     }
 
     /**
@@ -361,43 +327,6 @@ public class ServiceDiscoveryRegistry implements Registry {
         return listener.getServiceNames() + ":" + url.toString(VERSION_KEY, GROUP_KEY, PROTOCOL_KEY);
     }
 
-    /**
-     * 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 subscribedURL, final NotifyListener listener) {
-        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)) {
-            Set<String> mappedServices = findMappedServices(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 " + getSubscribedServices() + " by default.");
-                subscribedServices.addAll(getSubscribedServices());
-            }
-        }
-        return subscribedServices;
-    }
-
-    public static Set<String> parseServices(String literalServices) {
-        return isBlank(literalServices) ? emptySet() :
-                unmodifiableSet(of(literalServices.split(","))
-                        .map(String::trim)
-                        .filter(StringUtils::isNotEmpty)
-                        .collect(toSet()));
-    }
-
 //    public void doSubscribe(URL url, NotifyListener listener) {
 //        writableMetadataService.subscribeURL(url);
 //
@@ -435,25 +364,6 @@ public class ServiceDiscoveryRegistry implements Registry {
 //    }
 
     /**
-     * Get the subscribed service names
-     *
-     * @return non-null
-     */
-    public Set<String> getSubscribedServices() {
-        return subscribedServices;
-    }
-
-    /**
-     * Get the mapped services name by the specified {@link URL}
-     *
-     * @param subscribedURL
-     * @return
-     */
-    protected Set<String> findMappedServices(URL subscribedURL, MappingListener listener) {
-        return serviceNameMapping.getAndListen(subscribedURL, listener);
-    }
-
-    /**
      * Create an instance of {@link ServiceDiscoveryRegistry} if supported
      *
      * @param registryURL the {@link URL url} of registry
@@ -500,40 +410,4 @@ public class ServiceDiscoveryRegistry implements Registry {
                 || Objects.equals(protocol, targetURL.getProtocol());
     }
 
-    private class DefaultMappingListener implements MappingListener {
-        private final Logger logger = LoggerFactory.getLogger(DefaultMappingListener.class);
-        private URL url;
-        private Set<String> oldApps;
-        private NotifyListener listener;
-
-        public DefaultMappingListener(URL subscribedURL, Set<String> serviceNames, NotifyListener listener) {
-            this.url = subscribedURL;
-            this.oldApps = serviceNames;
-            this.listener = listener;
-        }
-
-        @Override
-        public void onEvent(MappingChangedEvent event) {
-            logger.info("Received mapping notification from meta server, " +  event);
-            Set<String> newApps = event.getApps();
-            Set<String> tempOldApps = oldApps;
-            oldApps = newApps;
-
-            if (CollectionUtils.isEmpty(newApps)) {
-                return;
-            }
-
-            if (CollectionUtils.isEmpty(tempOldApps) && newApps.size() > 0) {
-                subscribeURLs(url, listener, newApps);
-                return;
-            }
-
-            for (String newAppName : newApps) {
-                if (!tempOldApps.contains(newAppName)) {
-                    subscribeURLs(url, listener, newApps);
-                    return;
-                }
-            }
-        }
-    }
 }
\ No newline at end of file
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/event/listener/ServiceInstancesChangedListener.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/event/listener/ServiceInstancesChangedListener.java
index f0fcd33..4a4e634 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/event/listener/ServiceInstancesChangedListener.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/event/listener/ServiceInstancesChangedListener.java
@@ -111,7 +111,7 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
         }
 
         Map<String, List<ServiceInstance>> revisionToInstances = new HashMap<>();
-        Map<String, Set<String>> localServiceToRevisions = new HashMap<>();
+        Map<ServiceInfo, Set<String>> localServiceToRevisions = new HashMap<>();
         Map<Set<String>, List<URL>> revisionsToUrls = new HashMap<>();
         Map<String, List<URL>> newServiceUrls = new HashMap<>();//TODO
         Map<String, MetadataInfo> newRevisionToMetadata = new HashMap<>();
@@ -139,7 +139,7 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
 
         if (hasEmptyMetadata(newRevisionToMetadata)) {// retry every 10 seconds
             if (retryPermission.tryAcquire()) {
-                scheduler.scheduleAtFixedRate(new AddressRefreshRetryTask(retryPermission), 10000, 10000, TimeUnit.MILLISECONDS);
+                scheduler.schedule(new AddressRefreshRetryTask(retryPermission), 10000, TimeUnit.MILLISECONDS);
                 logger.warn("Address refresh try task submitted.");
             }
             logger.warn("Address refresh failed because of Metadata Server failure, wait for retry or new address refresh event.");
@@ -149,19 +149,27 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
 
         this.revisionToMetadata = newRevisionToMetadata;
 
-        localServiceToRevisions.forEach((serviceKey, revisions) -> {
+        localServiceToRevisions.forEach((serviceInfo, revisions) -> {
             List<URL> urls = revisionsToUrls.get(revisions);
             if (urls != null) {
-                newServiceUrls.put(serviceKey, urls);
+                newServiceUrls.put(serviceInfo.getMatchKey(), urls);
             } else {
                 urls = new ArrayList<>();
                 for (String r : revisions) {
                     for (ServiceInstance i : revisionToInstances.get(r)) {
+                        // different protocols may have ports specified in meta
+                        if (ServiceInstanceMetadataUtils.hasEndpoints(i)) {
+                            DefaultServiceInstance.Endpoint endpoint = ServiceInstanceMetadataUtils.getEndpoint(i, serviceInfo.getProtocol());
+                            if (endpoint != null && !endpoint.getPort().equals(i.getPort())) {
+                                urls.add(((DefaultServiceInstance)i).copy(endpoint).toURL());
+                                break;
+                            }
+                        }
                         urls.add(i.toURL());
                     }
                 }
                 revisionsToUrls.put(revisions, urls);
-                newServiceUrls.put(serviceKey, urls);
+                newServiceUrls.put(serviceInfo.getMatchKey(), urls);
             }
         });
         this.serviceUrls = newServiceUrls;
@@ -169,7 +177,7 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
         this.notifyAddressChanged();
     }
 
-    public synchronized void addListener(String serviceKey, NotifyListener listener) {
+    public synchronized void addListenerAndNotify(String serviceKey, NotifyListener listener) {
         this.listeners.put(serviceKey, listener);
         List<URL> urls = this.serviceUrls.get(serviceKey);
         if (CollectionUtils.isNotEmpty(urls)) {
@@ -248,10 +256,10 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
         return result;
     }
 
-    private MetadataInfo getRemoteMetadata(ServiceInstance instance, String revision, Map<String, Set<String>> localServiceToRevisions, List<ServiceInstance> subInstances) {
+    private MetadataInfo getRemoteMetadata(ServiceInstance instance, String revision, Map<ServiceInfo, Set<String>> localServiceToRevisions, List<ServiceInstance> subInstances) {
         MetadataInfo metadata = revisionToMetadata.get(revision);
         if (metadata == null) {
-            if (failureCounter.get() < 3 || (System.currentTimeMillis() - lastFailureTime > 5000)) {
+            if (failureCounter.get() < 3 || (System.currentTimeMillis() - lastFailureTime > 10000)) {
                 metadata = getMetadataInfo(instance);
                 if (metadata != null) {
                     logger.info("MetadataInfo for instance " + instance.getAddress() + "?revision=" + revision + " is " + metadata);
@@ -260,7 +268,7 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
                     parseMetadata(revision, metadata, localServiceToRevisions);
                 } else {
                     logger.error("Failed to get MetadataInfo for instance " + instance.getAddress() + "?revision=" + revision
-                            + ", async task added, wait for retry.");
+                            + ", wait for retry.");
                     lastFailureTime = System.currentTimeMillis();
                     failureCounter.incrementAndGet();
                 }
@@ -272,10 +280,10 @@ public class ServiceInstancesChangedListener implements ConditionalEventListener
         return metadata;
     }
 
-    private Map<String, Set<String>> parseMetadata(String revision, MetadataInfo metadata, Map<String, Set<String>> localServiceToRevisions) {
+    private Map<ServiceInfo, Set<String>> parseMetadata(String revision, MetadataInfo metadata, Map<ServiceInfo, Set<String>> localServiceToRevisions) {
         Map<String, ServiceInfo> serviceInfos = metadata.getServices();
         for (Map.Entry<String, ServiceInfo> entry : serviceInfos.entrySet()) {
-            Set<String> set = localServiceToRevisions.computeIfAbsent(entry.getKey(), k -> new TreeSet<>());
+            Set<String> set = localServiceToRevisions.computeIfAbsent(entry.getValue(), k -> new TreeSet<>());
             set.add(revision);
         }
 
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceURLParamsMetadataCustomizer.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceURLParamsMetadataCustomizer.java
index 9cc28a7..e04aff4 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceURLParamsMetadataCustomizer.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataServiceURLParamsMetadataCustomizer.java
@@ -16,16 +16,13 @@
  */
 package org.apache.dubbo.registry.client.metadata;
 
-import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.WritableMetadataService;
 import org.apache.dubbo.registry.client.ServiceInstance;
 import org.apache.dubbo.registry.client.ServiceInstanceCustomizer;
 
 import java.util.Map;
-import java.util.SortedSet;
 
 import static org.apache.dubbo.common.utils.StringUtils.isBlank;
-import static org.apache.dubbo.metadata.MetadataService.toURLs;
 import static org.apache.dubbo.metadata.WritableMetadataService.getDefaultExtension;
 import static org.apache.dubbo.registry.client.metadata.ServiceInstanceMetadataUtils.METADATA_SERVICE_URL_PARAMS_PROPERTY_NAME;
 import static org.apache.dubbo.registry.client.metadata.ServiceInstanceMetadataUtils.getMetadataServiceParameter;
@@ -51,15 +48,6 @@ public class MetadataServiceURLParamsMetadataCustomizer implements ServiceInstan
 
     private String resolveMetadataPropertyValue(ServiceInstance serviceInstance) {
         WritableMetadataService writableMetadataService = getDefaultExtension();
-
-        String serviceInterface = MetadataService.class.getName();
-
-        String group = serviceInstance.getServiceName();
-
-        String version = MetadataService.VERSION;
-
-        SortedSet<String> urls = writableMetadataService.getExportedURLs(serviceInterface, group, version);
-
-        return getMetadataServiceParameter(toURLs(urls));
+        return getMetadataServiceParameter(writableMetadataService.getMetadataServiceURL());
     }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataUtils.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataUtils.java
index 0a6a106..0321a16 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataUtils.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/MetadataUtils.java
@@ -50,11 +50,18 @@ public class MetadataUtils {
 
     public static WritableMetadataService localMetadataService;
 
+    public static WritableMetadataService getLocalMetadataService() {
+        if (localMetadataService == null) {
+            localMetadataService = WritableMetadataService.getDefaultExtension();
+        }
+        return localMetadataService;
+    }
+
     public static RemoteMetadataServiceImpl getRemoteMetadataService() {
         if (remoteMetadataService == null) {
             synchronized (REMOTE_LOCK) {
                 if (remoteMetadataService == null) {
-                    remoteMetadataService = new RemoteMetadataServiceImpl(WritableMetadataService.getDefaultExtension());
+                    remoteMetadataService = new RemoteMetadataServiceImpl(getLocalMetadataService());
                 }
             }
         }
@@ -99,4 +106,9 @@ public class MetadataUtils {
             return proxyFactory.getProxy(invoker);
         });
     }
+
+    public static void saveMetadataURL(URL url) {
+        // store in local
+        getLocalMetadataService().setMetadataServiceURL(url);
+    }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ProtocolPortsMetadataCustomizer.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ProtocolPortsMetadataCustomizer.java
index c8dffc4..12099ce 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ProtocolPortsMetadataCustomizer.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ProtocolPortsMetadataCustomizer.java
@@ -16,8 +16,6 @@
  */
 package org.apache.dubbo.registry.client.metadata;
 
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.WritableMetadataService;
 import org.apache.dubbo.registry.client.ServiceInstance;
 import org.apache.dubbo.registry.client.ServiceInstanceCustomizer;
@@ -41,15 +39,14 @@ public class ProtocolPortsMetadataCustomizer implements ServiceInstanceCustomize
         WritableMetadataService writableMetadataService = WritableMetadataService.getDefaultExtension();
 
         Map<String, Integer> protocols = new HashMap<>();
-        writableMetadataService.getExportedURLs()
-                .stream()
-                .map(URL::valueOf)
-                .filter(url -> !MetadataService.class.getName().equals(url.getServiceInterface()))
+        writableMetadataService.getExportedServiceURLs()
                 .forEach(url -> {
                     // TODO, same protocol listen on different ports will override with each other.
                     protocols.put(url.getProtocol(), url.getPort());
                 });
 
-        setEndpoints(serviceInstance, protocols);
+        if (protocols.size() > 0) {// set endpoints only for multi-protocol scenario
+            setEndpoints(serviceInstance, protocols);
+        }
     }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataCustomizer.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataCustomizer.java
index 8b8a5b7..92cbead 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataCustomizer.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataCustomizer.java
@@ -38,9 +38,14 @@ import static org.apache.dubbo.common.constants.CommonConstants.APPLICATION_KEY;
  */
 public class ServiceInstanceMetadataCustomizer implements ServiceInstanceCustomizer {
 
+    private volatile boolean customized = false;
+
     @Override
     public void customize(ServiceInstance serviceInstance) {
-        Map<String, String> params = new HashMap<>();
+        if (customized) {
+            return;
+        }
+        customized = true;
 
         ExtensionLoader<MetadataParamsFilter> loader = ExtensionLoader.getExtensionLoader(MetadataParamsFilter.class);
         Set<MetadataParamsFilter> paramsFilters = loader.getSupportedExtensionInstances();
@@ -49,7 +54,14 @@ public class ServiceInstanceMetadataCustomizer implements ServiceInstanceCustomi
                 = (InMemoryWritableMetadataService) WritableMetadataService.getDefaultExtension();
         // pick the first interface metadata available.
         // FIXME, check the same key in different urls has the same value
-        MetadataInfo metadataInfo = localMetadataService.getMetadataInfos().values().iterator().next();
+        Map<String, MetadataInfo> metadataInfos = localMetadataService.getMetadataInfos();
+        if (CollectionUtils.isEmptyMap(metadataInfos)) {
+            return;
+        }
+        MetadataInfo metadataInfo = metadataInfos.values().iterator().next();
+        if (metadataInfo == null || CollectionUtils.isEmptyMap(metadataInfo.getServices())) {
+            return;
+        }
         MetadataInfo.ServiceInfo serviceInfo = metadataInfo.getServices().values().iterator().next();
         Map<String, String> allParams = new HashMap<>(serviceInfo.getUrl().getParameters());
 
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtils.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtils.java
index 95e5349..acfa492 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtils.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtils.java
@@ -17,12 +17,18 @@
 package org.apache.dubbo.registry.client.metadata;
 
 import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.extension.ExtensionLoader;
+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.MetadataInfo;
 import org.apache.dubbo.metadata.MetadataService;
 import org.apache.dubbo.metadata.WritableMetadataService;
+import org.apache.dubbo.registry.client.DefaultServiceInstance;
+import org.apache.dubbo.registry.client.DefaultServiceInstance.Endpoint;
 import org.apache.dubbo.registry.client.ServiceDiscovery;
 import org.apache.dubbo.registry.client.ServiceInstance;
+import org.apache.dubbo.registry.client.ServiceInstanceCustomizer;
 import org.apache.dubbo.registry.client.metadata.store.RemoteMetadataServiceImpl;
 import org.apache.dubbo.registry.support.AbstractRegistryFactory;
 import org.apache.dubbo.rpc.model.ApplicationModel;
@@ -30,7 +36,6 @@ import org.apache.dubbo.rpc.model.ApplicationModel;
 import com.alibaba.fastjson.JSON;
 
 import java.util.ArrayList;
-import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -40,6 +45,7 @@ import static org.apache.dubbo.common.constants.CommonConstants.APPLICATION_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.DEFAULT_METADATA_STORAGE_TYPE;
 import static org.apache.dubbo.common.constants.CommonConstants.GROUP_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.PORT_KEY;
+import static org.apache.dubbo.common.constants.CommonConstants.PROTOCOL_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.TIMESTAMP_KEY;
 import static org.apache.dubbo.common.utils.StringUtils.isBlank;
 import static org.apache.dubbo.registry.integration.InterfaceCompatibleRegistryProtocol.DEFAULT_REGISTER_PROVIDER_KEYS;
@@ -56,6 +62,7 @@ import static org.apache.dubbo.rpc.Constants.ID_KEY;
  * @since 2.7.5
  */
 public class ServiceInstanceMetadataUtils {
+    private static final Logger LOGGER = LoggerFactory.getLogger(ServiceInstanceMetadataUtils.class);
 
     /**
      * The prefix of {@link MetadataService} : "dubbo.metadata-service."
@@ -95,42 +102,19 @@ public class ServiceInstanceMetadataUtils {
      *
      * @param serviceInstance the instance of {@link ServiceInstance}
      * @return non-null {@link Map}, the key is {@link URL#getProtocol() the protocol of URL}, the value is
-     * {@link #getMetadataServiceURLParams(ServiceInstance, String)}
      */
-    public static Map<String, Map<String, String>> getMetadataServiceURLsParams(ServiceInstance serviceInstance) {
+    public static Map<String, String> getMetadataServiceURLsParams(ServiceInstance serviceInstance) {
         Map<String, String> metadata = serviceInstance.getMetadata();
         String param = metadata.get(METADATA_SERVICE_URL_PARAMS_PROPERTY_NAME);
         return isBlank(param) ? emptyMap() : (Map) JSON.parse(param);
     }
 
-    /**
-     * Get the {@link URL url's} parameters of {@link MetadataService MetadataService's} Metadata
-     *
-     * @param serviceInstance the instance of {@link ServiceInstance}
-     * @return non-null {@link Map}
-     */
-    public static Map<String, String> getMetadataServiceURLParams(ServiceInstance serviceInstance, String protocol) {
-        Map<String, Map<String, String>> params = getMetadataServiceURLsParams(serviceInstance);
-        return params.getOrDefault(protocol, emptyMap());
-    }
-
-    public static String getMetadataServiceParameter(List<URL> urls) {
-
-        Map<String, Map<String, String>> params = new HashMap<>();
-
-        urls.stream()
-                // remove APPLICATION_KEY because service name must be present
-                .map(url -> url.removeParameter(APPLICATION_KEY))
-                // remove GROUP_KEY, always uses application name.
-                .map(url -> url.removeParameter(GROUP_KEY))
-                // remove DEPRECATED_KEY because it's always false
-                .map(url -> url.removeParameter(DEPRECATED_KEY))
-                // remove TIMESTAMP_KEY because it's nonsense
-                .map(url -> url.removeParameter(TIMESTAMP_KEY))
-                .forEach(url -> {
-                    String protocol = url.getProtocol();
-                    params.put(protocol, getParams(url));
-                });
+    public static String getMetadataServiceParameter(URL url) {
+        url = url.removeParameter(APPLICATION_KEY);
+        url = url.removeParameter(GROUP_KEY);
+        url = url.removeParameter(DEPRECATED_KEY);
+        url = url.removeParameter(TIMESTAMP_KEY);
+        Map<String, String> params = getParams(url);
 
         if (params.isEmpty()) {
             return null;
@@ -143,6 +127,7 @@ public class ServiceInstanceMetadataUtils {
         Map<String, String> params = new LinkedHashMap<>();
         setDefaultParams(params, providerURL);
         params.put(PORT_KEY, String.valueOf(providerURL.getPort()));
+        params.put(PROTOCOL_KEY, providerURL.getProtocol());
         return params;
     }
 
@@ -194,6 +179,10 @@ public class ServiceInstanceMetadataUtils {
         return metadata.get(METADATA_CLUSTER_PROPERTY_NAME);
     }
 
+    public static boolean hasEndpoints(ServiceInstance serviceInstance) {
+        return StringUtils.isNotEmpty(serviceInstance.getMetadata().get(ENDPOINTS));
+    }
+
     /**
      * Is Dubbo Service instance or not
      *
@@ -225,14 +214,12 @@ public class ServiceInstanceMetadataUtils {
      * @param protocol        the name of protocol, e.g, dubbo, rest, and so on
      * @return if not found, return <code>null</code>
      */
-    public static Integer getProtocolPort(ServiceInstance serviceInstance, String protocol) {
-        Map<String, String> metadata = serviceInstance.getMetadata();
-        String rawEndpoints = metadata.get(ENDPOINTS);
-        if (StringUtils.isNotEmpty(rawEndpoints)) {
-            List<Endpoint> endpoints = JSON.parseArray(rawEndpoints, Endpoint.class);
+    public static Endpoint getEndpoint(ServiceInstance serviceInstance, String protocol) {
+        List<Endpoint> endpoints = ((DefaultServiceInstance)serviceInstance).getEndpoints();
+        if (endpoints != null) {
             for (Endpoint endpoint : endpoints) {
                 if (endpoint.getProtocol().equals(protocol)) {
-                    return endpoint.getPort();
+                    return endpoint;
                 }
             }
         }
@@ -260,14 +247,29 @@ public class ServiceInstanceMetadataUtils {
         return "true".equals(instance.getExtendParams().get(INSTANCE_REVISION_UPDATED_KEY));
     }
 
-    public static void refreshMetadataAndInstance() {
+    public static void refreshMetadataAndInstance(ServiceInstance serviceInstance) {
         RemoteMetadataServiceImpl remoteMetadataService = MetadataUtils.getRemoteMetadataService();
         remoteMetadataService.publishMetadata(ApplicationModel.getName());
 
         AbstractRegistryFactory.getServiceDiscoveries().forEach(serviceDiscovery -> {
-            calInstanceRevision(serviceDiscovery, serviceDiscovery.getLocalInstance());
+            ServiceInstance instance = serviceDiscovery.getLocalInstance() == null ? serviceInstance : serviceDiscovery.getLocalInstance();
+            if (instance == null) {
+                LOGGER.error("Error refreshing service instance, instance not registered yet.");
+            }
+            calInstanceRevision(serviceDiscovery, instance);
+            customizeInstance(instance);
             // update service instance revision
-            serviceDiscovery.update(serviceDiscovery.getLocalInstance());
+            serviceDiscovery.update(instance);
+        });
+    }
+
+    public static void customizeInstance(ServiceInstance instance) {
+        ExtensionLoader<ServiceInstanceCustomizer> loader =
+                ExtensionLoader.getExtensionLoader(ServiceInstanceCustomizer.class);
+        // FIXME, sort customizer before apply
+        loader.getSupportedExtensionInstances().forEach(customizer -> {
+            // customizes
+            customizer.customize(instance);
         });
     }
 
@@ -286,29 +288,4 @@ public class ServiceInstanceMetadataUtils {
         }
     }
 
-    public static class Endpoint {
-        Integer port;
-        String protocol;
-
-        public Endpoint(Integer port, String protocol) {
-            this.port = port;
-            this.protocol = protocol;
-        }
-
-        public Integer getPort() {
-            return port;
-        }
-
-        public void setPort(Integer port) {
-            this.port = port;
-        }
-
-        public String getProtocol() {
-            return protocol;
-        }
-
-        public void setProtocol(String protocol) {
-            this.protocol = protocol;
-        }
-    }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/StandardMetadataServiceURLBuilder.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/StandardMetadataServiceURLBuilder.java
index c90eea0..8adf71a 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/StandardMetadataServiceURLBuilder.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/StandardMetadataServiceURLBuilder.java
@@ -26,10 +26,10 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import static java.lang.String.valueOf;
 import static org.apache.dubbo.common.constants.CommonConstants.CONSUMER;
 import static org.apache.dubbo.common.constants.CommonConstants.GROUP_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.PORT_KEY;
+import static org.apache.dubbo.common.constants.CommonConstants.PROTOCOL_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.SIDE_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.TIMEOUT_KEY;
 import static org.apache.dubbo.metadata.MetadataConstants.DEFAULT_METADATA_TIMEOUT_VALUE;
@@ -54,36 +54,31 @@ public class StandardMetadataServiceURLBuilder implements MetadataServiceURLBuil
      */
     @Override
     public List<URL> build(ServiceInstance serviceInstance) {
-
-        Map<String, Map<String, String>> paramsMap = getMetadataServiceURLsParams(serviceInstance);
+        Map<String, String> paramsMap = getMetadataServiceURLsParams(serviceInstance);
 
         List<URL> urls = new ArrayList<>(paramsMap.size());
 
         String serviceName = serviceInstance.getServiceName();
 
         String host = serviceInstance.getHost();
-
-        for (Map.Entry<String, Map<String, String>> entry : paramsMap.entrySet()) {
-            String protocol = entry.getKey();
-            Map<String, String> params = entry.getValue();
-            int port = Integer.parseInt(params.get(PORT_KEY));
-            URLBuilder urlBuilder = new URLBuilder()
-                    .setHost(host)
-                    .setPort(port)
-                    .setProtocol(protocol)
-                    .setPath(MetadataService.class.getName())
-                    .addParameter(TIMEOUT_KEY, ConfigurationUtils.get(METADATA_PROXY_TIMEOUT_KEY, DEFAULT_METADATA_TIMEOUT_VALUE))
-                    .addParameter(SIDE_KEY, CONSUMER);
-
-            // add parameters
-            params.forEach((name, value) -> urlBuilder.addParameter(name, valueOf(value)));
-
-            // add the default parameters
-            urlBuilder.addParameter(GROUP_KEY, serviceName);
-
-            urls.add(urlBuilder.build());
+        URLBuilder urlBuilder = new URLBuilder();
+        for (Map.Entry<String, String> entry : paramsMap.entrySet()) {
+            String key = entry.getKey();
+            String value = entry.getValue();
+            if (key.equals(PORT_KEY)) {
+                urlBuilder.setPort(Integer.parseInt(value));
+            } else if (key.equals(PROTOCOL_KEY)) {
+                urlBuilder.setProtocol(value);
+            } else {
+                urlBuilder.addParameter(key, value);
+            }
         }
+        urlBuilder.setHost(host).setPath(MetadataService.class.getName())
+                .addParameter(TIMEOUT_KEY, ConfigurationUtils.get(METADATA_PROXY_TIMEOUT_KEY, DEFAULT_METADATA_TIMEOUT_VALUE))
+                .addParameter(SIDE_KEY, CONSUMER)
+                .addParameter(GROUP_KEY, serviceName);
 
+        urls.add(urlBuilder.build());
         return urls;
     }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/store/InMemoryWritableMetadataService.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/store/InMemoryWritableMetadataService.java
index c2f03ee..736f413 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/store/InMemoryWritableMetadataService.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/metadata/store/InMemoryWritableMetadataService.java
@@ -23,6 +23,7 @@ import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.metadata.MetadataInfo;
 import org.apache.dubbo.metadata.MetadataInfo.ServiceInfo;
 import org.apache.dubbo.metadata.MetadataService;
+import org.apache.dubbo.metadata.ServiceNameMapping;
 import org.apache.dubbo.metadata.WritableMetadataService;
 import org.apache.dubbo.metadata.definition.ServiceDefinitionBuilder;
 import org.apache.dubbo.metadata.definition.model.ServiceDefinition;
@@ -33,7 +34,10 @@ import org.apache.dubbo.rpc.support.ProtocolUtils;
 import com.google.gson.Gson;
 
 import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 import java.util.SortedSet;
 import java.util.TreeSet;
 import java.util.concurrent.Callable;
@@ -73,8 +77,11 @@ public class InMemoryWritableMetadataService implements WritableMetadataService
      * and value is the {@link SortedSet sorted set} of the {@link URL URLs}
      */
     ConcurrentNavigableMap<String, SortedSet<URL>> exportedServiceURLs = new ConcurrentSkipListMap<>();
+    URL metadataServiceURL;
     ConcurrentMap<String, MetadataInfo> metadataInfos;
-    final Semaphore metadataSemaphore = new Semaphore(1);
+    final Semaphore metadataSemaphore = new Semaphore(0);
+
+    final Map<String, Set<String>> serviceToAppsMapping = new HashMap<>();
 
     // ==================================================================================== //
 
@@ -123,7 +130,20 @@ public class InMemoryWritableMetadataService implements WritableMetadataService
     }
 
     @Override
+    public Set<URL> getExportedServiceURLs() {
+        Set<URL> set = new HashSet<>();
+        for (Map.Entry<String, SortedSet<URL>> entry : exportedServiceURLs.entrySet()) {
+            set.addAll(entry.getValue());
+        }
+        return set;
+    }
+
+    @Override
     public boolean exportURL(URL url) {
+        if (MetadataService.class.getName().equals(url.getServiceInterface())) {
+            this.metadataServiceURL = url;
+            return true;
+        }
         String registryCluster = RegistryClusterIdentifier.getExtension(url).providerKey(url);
         String[] clusters = registryCluster.split(",");
         for (String cluster : clusters) {
@@ -138,14 +158,19 @@ public class InMemoryWritableMetadataService implements WritableMetadataService
 
     @Override
     public boolean unexportURL(URL url) {
+        if (MetadataService.class.getName().equals(url.getServiceInterface())) {
+            // TODO, metadata service need to be unexported.
+            this.metadataServiceURL = null;
+            return true;
+        }
         String registryCluster = RegistryClusterIdentifier.getExtension(url).providerKey(url);
         String[] clusters = registryCluster.split(",");
         for (String cluster : clusters) {
             MetadataInfo metadataInfo = metadataInfos.get(cluster);
             metadataInfo.removeService(url.getProtocolServiceKey());
-            if (metadataInfo.getServices().isEmpty()) {
-                metadataInfos.remove(cluster);
-            }
+//            if (metadataInfo.getServices().isEmpty()) {
+//                metadataInfos.remove(cluster);
+//            }
         }
         metadataSemaphore.release();
         return removeURL(exportedServiceURLs, url);
@@ -208,6 +233,7 @@ public class InMemoryWritableMetadataService implements WritableMetadataService
     public void blockUntilUpdated() {
         try {
             metadataSemaphore.acquire();
+            metadataSemaphore.drainPermits();
         } catch (InterruptedException e) {
             logger.warn("metadata refresh thread has been interrupted unexpectedly while wating for update.", e);
         }
@@ -217,6 +243,36 @@ public class InMemoryWritableMetadataService implements WritableMetadataService
         return metadataInfos;
     }
 
+    void addMetaServiceURL(URL url) {
+        this.metadataServiceURL = url;
+    }
+
+    @Override
+    public URL getMetadataServiceURL() {
+        return this.metadataServiceURL;
+    }
+
+    @Override
+    public void putCachedMapping(String serviceKey, Set<String> apps) {
+        serviceToAppsMapping.put(serviceKey, apps);
+    }
+
+    @Override
+    public Set<String> getCachedMapping(URL consumerURL) {
+        String serviceKey = ServiceNameMapping.buildMappingKey(consumerURL);
+        return serviceToAppsMapping.get(serviceKey);
+    }
+
+    @Override
+    public Set<String> removeCachedMapping(String serviceKey) {
+        return serviceToAppsMapping.remove(serviceKey);
+    }
+
+    @Override
+    public void setMetadataServiceURL(URL url) {
+        this.metadataServiceURL = url;
+    }
+
     boolean addURL(Map<String, SortedSet<URL>> serviceURLs, URL url) {
         return executeMutually(() -> {
             SortedSet<URL> urls = serviceURLs.computeIfAbsent(url.getServiceKey(), this::newSortedURLs);
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 aaa3b3b..30f1ea1 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
@@ -21,6 +21,7 @@ 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.WritableMetadataService;
 import org.apache.dubbo.registry.client.migration.model.MigrationRule;
 import org.apache.dubbo.rpc.Invoker;
 import org.apache.dubbo.rpc.cluster.ClusterInvoker;
@@ -33,6 +34,8 @@ public class DefaultMigrationAddressComparator implements MigrationAddressCompar
     private static final String DEFAULT_THRESHOLD_STRING = "0.8";
     private static final float DEFAULT_THREAD = 0.8f;
 
+    private static final WritableMetadataService localMetadataService = WritableMetadataService.getDefaultExtension();
+
     @Override
     public <T> boolean shouldMigrate(ClusterInvoker<T> serviceDiscoveryInvoker, ClusterInvoker<T> invoker, MigrationRule rule) {
         if (!serviceDiscoveryInvoker.hasProxyInvokers()) {
@@ -51,7 +54,8 @@ public class DefaultMigrationAddressComparator implements MigrationAddressCompar
         int oldAddressSize = CollectionUtils.isNotEmpty(invokers2) ? invokers2.size() : 0;
 
         String rawThreshold = null;
-        Float configedThreshold = rule == null ? null : rule.getThreshold(invoker.getUrl().getServiceKey());
+        String serviceKey = invoker.getUrl().getDisplayServiceKey();
+        Float configedThreshold = rule == null ? null : rule.getThreshold(serviceKey, localMetadataService.getCachedMapping(invoker.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/MigrationClusterInvoker.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationClusterInvoker.java
index d1dca9d..91fa942 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationClusterInvoker.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationClusterInvoker.java
@@ -44,5 +44,7 @@ public interface MigrationClusterInvoker<T> extends ClusterInvoker<T> {
 
     void migrateToServiceDiscoveryInvoker(boolean forceMigrate);
 
+    void refreshServiceDiscoveryInvokerOnMappingCallback(boolean forceMigrate);
+
     void reRefer(URL newSubscribeUrl);
 }
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 28b49ce..15e2434 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
@@ -160,6 +160,16 @@ public class MigrationInvoker<T> implements MigrationClusterInvoker<T> {
     }
 
     @Override
+    public void refreshServiceDiscoveryInvokerOnMappingCallback(boolean forceMigrate) {
+        if (this.serviceDiscoveryInvoker != null) {
+            DynamicDirectory dynamicDirectory = (DynamicDirectory) this.serviceDiscoveryInvoker.getDirectory();
+            dynamicDirectory.subscribe(dynamicDirectory.getOriginalConsumerUrl());
+        } else {
+            migrateToServiceDiscoveryInvoker(forceMigrate);
+        }
+    }
+
+    @Override
     public Result invoke(Invocation invocation) throws RpcException {
         if (currentAvailableInvoker != null) {
             return currentAvailableInvoker.invoke(invocation);
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 9c4071b..5972e06 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
@@ -22,11 +22,11 @@ import org.apache.dubbo.common.extension.Activate;
 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.WritableMetadataService;
 import org.apache.dubbo.registry.client.migration.model.MigrationRule;
 import org.apache.dubbo.registry.client.migration.model.MigrationStep;
 
-import static org.apache.dubbo.common.constants.RegistryConstants.INIT;
+import java.util.Set;
 
 @Activate
 public class MigrationRuleHandler<T> {
@@ -36,48 +36,68 @@ public class MigrationRuleHandler<T> {
     private MigrationClusterInvoker<T> migrationInvoker;
     private MigrationStep currentStep;
     private Float currentThreshold = 0f;
-    private MigrationRule rule;
     private URL consumerURL;
 
+    private final WritableMetadataService writableMetadataService;
+
     public MigrationRuleHandler(MigrationClusterInvoker<T> invoker, URL url) {
         this.migrationInvoker = invoker;
         this.consumerURL = url;
+        this.writableMetadataService = WritableMetadataService.getDefaultExtension();
     }
 
-    public void doMigrate(String rawRule) {
-        MigrationStep step = (migrationInvoker instanceof ServiceDiscoveryMigrationInvoker)
-                ? MigrationStep.FORCE_APPLICATION
-                : MigrationStep.INTERFACE_FIRST;
-        Float threshold = -1f;
-        if (StringUtils.isEmpty(rawRule)) {
-            logger.error("Find empty migration rule, will ignore.");
+    public synchronized void doMigrate(MigrationRule rule, boolean isCallback) {
+        if (migrationInvoker instanceof ServiceDiscoveryMigrationInvoker) {
+            if (!isCallback) {
+                initInvoker(MigrationStep.FORCE_APPLICATION, 1.0f);
+            } else {
+                migrationInvoker.refreshServiceDiscoveryInvokerOnMappingCallback(true);
+            }
             return;
-        } else if (INIT.equals(rawRule)) {
+        }
+
+        MigrationStep step = MigrationStep.INTERFACE_FIRST;
+        Float threshold = -1f;
+        if (rule == MigrationRule.INIT) {
             step = Enum.valueOf(MigrationStep.class, ConfigurationUtils.getDynamicProperty(DUBBO_SERVICEDISCOVERY_MIGRATION, step.name()));
         } else {
             try {
-                rule = MigrationRule.parse(rawRule);
+                String serviceKey = consumerURL.getDisplayServiceKey();
+                Set<String> apps = writableMetadataService.getCachedMapping(consumerURL);
                 // FIXME, consumerURL.getHost() might not exactly the ip expected.
-                if (CollectionUtils.isEmpty(rule.getTargetIps())) {
-                    setMigrationRule(rule);
-                    step = rule.getStep(consumerURL.getServiceKey());
-                    threshold = rule.getThreshold(consumerURL.getServiceKey());
-                } else {
-                    if (rule.getTargetIps().contains(consumerURL.getHost())) {
+                if (CollectionUtils.isNotEmpty(apps)) { //empty only happens when meta server does not work properly
+                    if (CollectionUtils.isEmpty(rule.getTargetIps())) {
                         setMigrationRule(rule);
-                        step = rule.getStep(consumerURL.getServiceKey());
-                        threshold = rule.getThreshold(consumerURL.getServiceKey());
+                        step = getMigrationStep(rule, step, serviceKey, apps);
+                        threshold = getMigrationThreshold(rule, threshold, serviceKey, apps);
                     } else {
-                        setMigrationRule(null); // clear previous rule
-                        logger.info("New migration rule ignored and previous migration rule cleared, new target ips " + rule.getTargetIps() + " and local ip " + consumerURL.getHost() + " do not match");
+                        if (rule.getTargetIps().contains(consumerURL.getHost())) {
+                            setMigrationRule(rule);
+                            step = getMigrationStep(rule, step, serviceKey, apps);
+                            threshold = getMigrationThreshold(rule, threshold, serviceKey, apps);
+                        } else {
+                            setMigrationRule(null); // clear previous rule
+                            logger.info("New migration rule ignored and previous migration rule cleared, new target ips " + rule.getTargetIps() + " and local ip " + consumerURL.getHost() + " do not match");
+                        }
                     }
                 }
             } catch (Exception e) {
-                logger.error("Parse migration rule error, will use default step " + step, e);
+                logger.error("Failed to get step and threshold info from rule: " + rule, e);
             }
         }
 
-        if ((currentStep == null || currentStep != step) || (!currentThreshold.equals(threshold))) {
+        if (!isCallback) {
+            initInvoker(step, threshold);
+        } else {
+            refreshInvoker(step, threshold);
+        }
+    }
+
+    private void initInvoker(MigrationStep step, Float threshold) {
+        if (step == null || threshold == null) {
+            throw new IllegalStateException("Step or threshold of migration rule cannot be null");
+        }
+        if ((currentStep == null || currentStep != step) || !currentThreshold.equals(threshold)) {
             setCurrentStepAndThreshold(step, threshold);
             switch (step) {
                 case APPLICATION_FIRST:
@@ -93,6 +113,34 @@ public class MigrationRuleHandler<T> {
         }
     }
 
+    private void refreshInvoker(MigrationStep step, Float threshold) {
+        if (step == null || threshold == null) {
+            throw new IllegalStateException("Step or threshold of migration rule cannot be null");
+        }
+
+        if (step == MigrationStep.APPLICATION_FIRST) {
+            migrationInvoker.refreshServiceDiscoveryInvokerOnMappingCallback(false);
+        } else if (step == MigrationStep.FORCE_APPLICATION) {
+            migrationInvoker.refreshServiceDiscoveryInvokerOnMappingCallback(true);
+        }
+    }
+
+    public void setMigrationRule(MigrationRule rule) {
+        this.migrationInvoker.setMigrationRule(rule);
+    }
+
+    private MigrationStep getMigrationStep(MigrationRule rule, MigrationStep step, String serviceKey, Set<String> apps) {
+        MigrationStep configuredStep = rule.getStep(serviceKey, apps);
+        step = configuredStep == null ? step : configuredStep;
+        return step;
+    }
+
+    private Float getMigrationThreshold(MigrationRule rule, Float threshold, String serviceKey, Set<String> apps) {
+        Float configuredThreshold = rule.getThreshold(serviceKey, apps);
+        threshold = configuredThreshold == null ? threshold : configuredThreshold;
+        return threshold;
+    }
+
     public void setCurrentStepAndThreshold(MigrationStep currentStep, Float currentThreshold) {
         if (currentThreshold != null) {
             this.currentThreshold = currentThreshold;
@@ -102,8 +150,4 @@ public class MigrationRuleHandler<T> {
             this.migrationInvoker.setMigrationStep(currentStep);
         }
     }
-
-    public void setMigrationRule(MigrationRule rule) {
-        this.migrationInvoker.setMigrationRule(rule);
-    }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleListener.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleListener.java
index c667376..45a3156 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleListener.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/MigrationRuleListener.java
@@ -24,17 +24,33 @@ import org.apache.dubbo.common.extension.Activate;
 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.ConcurrentHashSet;
 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.WritableMetadataService;
+import org.apache.dubbo.registry.client.migration.model.MigrationRule;
 import org.apache.dubbo.registry.integration.RegistryProtocol;
 import org.apache.dubbo.registry.integration.RegistryProtocolListener;
 import org.apache.dubbo.rpc.Exporter;
 import org.apache.dubbo.rpc.cluster.ClusterInvoker;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 
+import java.util.Map;
 import java.util.Set;
-
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentHashMap;
+
+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.MAPPING_KEY;
 import static org.apache.dubbo.common.constants.RegistryConstants.INIT;
+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;
 
 @Activate
 public class MigrationRuleListener implements RegistryProtocolListener, ConfigurationListener {
@@ -42,10 +58,11 @@ public class MigrationRuleListener implements RegistryProtocolListener, Configur
     private static final String RULE_KEY = ApplicationModel.getName() + ".migration";
     private static final String DUBBO_SERVICEDISCOVERY_MIGRATION = "DUBBO_SERVICEDISCOVERY_MIGRATION";
 
-    private Set<MigrationRuleHandler> handlers = new ConcurrentHashSet<>();
+    private Map<String, MigrationRuleHandler> handlers = new ConcurrentHashMap<>();
     private DynamicConfiguration configuration;
 
     private volatile String rawRule;
+    private volatile MigrationRule rule;
 
     public MigrationRuleListener() {
         this.configuration = ApplicationModel.getEnvironment().getDynamicConfiguration().orElse(null);
@@ -64,7 +81,7 @@ public class MigrationRuleListener implements RegistryProtocolListener, Configur
             }
             rawRule = INIT;
         }
-        process(new ConfigChangedEvent(RULE_KEY, DUBBO_SERVICEDISCOVERY_MIGRATION, rawRule));
+//        process(new ConfigChangedEvent(RULE_KEY, DUBBO_SERVICEDISCOVERY_MIGRATION, rawRule));
     }
 
     @Override
@@ -78,9 +95,25 @@ public class MigrationRuleListener implements RegistryProtocolListener, Configur
         logger.info("Using the following migration rule to migrate:");
         logger.info(rawRule);
 
-        if (CollectionUtils.isNotEmpty(handlers)) {
-            handlers.forEach(listener -> listener.doMigrate(rawRule));
+        rule = parseRule(rawRule);
+
+        if (CollectionUtils.isNotEmptyMap(handlers)) {
+            handlers.forEach((_key, handler) -> handler.doMigrate(rule, false));
+        }
+    }
+
+    private MigrationRule parseRule(String rawRule) {
+        MigrationRule tmpRule = rule;
+        if (INIT.equals(rawRule)) {
+            tmpRule = MigrationRule.INIT;
+        } else {
+            try {
+                tmpRule = MigrationRule.parse(rawRule);
+            } catch (Exception e) {
+                logger.error("Failed to parse migration rule...", e);
+            }
         }
+        return tmpRule;
     }
 
     @Override
@@ -89,17 +122,106 @@ public class MigrationRuleListener implements RegistryProtocolListener, Configur
     }
 
     @Override
-    public synchronized void onRefer(RegistryProtocol registryProtocol, ClusterInvoker<?> invoker, URL url) {
-        MigrationInvoker<?> migrationInvoker = (MigrationInvoker<?>) invoker;
+    public synchronized void onRefer(RegistryProtocol registryProtocol, ClusterInvoker<?> invoker, URL consumerUrl, URL registryURL) {
+        MigrationRuleHandler<?> migrationRuleHandler = handlers.computeIfAbsent(consumerUrl.getServiceKey(), _key -> {
+            return new MigrationRuleHandler<>((MigrationInvoker<?>)invoker, consumerUrl);
+        });
+
+        try {
+            Set<String> subscribedServices = getServices(registryURL, consumerUrl, migrationRuleHandler);
+            WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(consumerUrl), subscribedServices);
+        } catch (Exception e) {
+            logger.warn("Cannot find app mapping for service " + consumerUrl.getServiceInterface() + ", will not migrate.", e);
+        }
 
-        MigrationRuleHandler<?> migrationRuleHandler = new MigrationRuleHandler<>(migrationInvoker, url);
-        handlers.add(migrationRuleHandler);
+        rule = parseRule(rawRule);
 
-        migrationRuleHandler.doMigrate(rawRule);
+        migrationRuleHandler.doMigrate(rule, false);
     }
 
     @Override
     public void onDestroy() {
         configuration.removeListener(RULE_KEY, this);
     }
+
+    /**
+     * 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, MigrationRuleHandler handler) {
+        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, handler));
+            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) {
+        return ServiceNameMapping.getExtension(registryURL.getParameter(MAPPING_KEY)).getAndListen(subscribedURL, listener);
+    }
+
+    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;
+        private Set<String> oldApps;
+        private MigrationRuleHandler handler;
+
+        public DefaultMappingListener(URL subscribedURL, Set<String> serviceNames, MigrationRuleHandler handler) {
+            this.url = subscribedURL;
+            this.oldApps = serviceNames;
+            this.handler = handler;
+        }
+
+        @Override
+        public void onEvent(MappingChangedEvent event) {
+            logger.info("Received mapping notification from meta server, " +  event);
+            Set<String> newApps = event.getApps();
+            Set<String> tempOldApps = oldApps;
+            oldApps = newApps;
+
+            if (CollectionUtils.isEmpty(newApps)) {
+                return;
+            }
+
+            if (CollectionUtils.isEmpty(tempOldApps) && newApps.size() > 0) {
+                WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(url), newApps);
+                handler.doMigrate(rule, true);
+                return;
+            }
+
+            for (String newAppName : newApps) {
+                if (!tempOldApps.contains(newAppName)) {
+                    WritableMetadataService.getDefaultExtension().putCachedMapping(ServiceNameMapping.buildMappingKey(url), newApps);
+                    handler.doMigrate(rule, true);
+                    return;
+                }
+            }
+        }
+    }
 }
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/ApplicationMigrationRule.java
similarity index 75%
copy from dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/InterfaceMigrationRule.java
copy to dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/client/migration/model/ApplicationMigrationRule.java
index 99ec43f..00e2a9f 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/ApplicationMigrationRule.java
@@ -16,17 +16,26 @@
  */
 package org.apache.dubbo.registry.client.migration.model;
 
-public class InterfaceMigrationRule {
-    private String serviceKey;
+public class ApplicationMigrationRule {
+    private String name;
     private MigrationStep step;
     private Float threshold;
 
-    public String getServiceKey() {
-        return serviceKey;
+    public ApplicationMigrationRule() {
     }
 
-    public void setServiceKey(String serviceKey) {
-        this.serviceKey = serviceKey;
+    public ApplicationMigrationRule(String name, MigrationStep step, Float threshold) {
+        this.name = name;
+        this.step = step;
+        this.threshold = threshold;
+    }
+
+    public String getName() {
+        return name;
+    }
+
+    public void setName(String name) {
+        this.name = name;
     }
 
     public MigrationStep getStep() {
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/InterfaceMigrationRule.java
index 99ec43f..a59b9d9 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/InterfaceMigrationRule.java
@@ -17,10 +17,28 @@
 package org.apache.dubbo.registry.client.migration.model;
 
 public class InterfaceMigrationRule {
+    private String appName;
     private String serviceKey;
     private MigrationStep step;
     private Float threshold;
 
+    public InterfaceMigrationRule(){}
+
+    public InterfaceMigrationRule(String appName, String serviceKey, MigrationStep step, Float threshold) {
+        this.appName = appName;
+        this.serviceKey = serviceKey;
+        this.step = step;
+        this.threshold = threshold;
+    }
+
+    public String getAppName() {
+        return appName;
+    }
+
+    public void setAppName(String appName) {
+        this.appName = appName;
+    }
+
     public String getServiceKey() {
         return serviceKey;
     }
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 c727627..69753ff 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,12 +16,16 @@
  */
 package org.apache.dubbo.registry.client.migration.model;
 
+import org.apache.dubbo.common.utils.CollectionUtils;
+
 import org.yaml.snakeyaml.Yaml;
 import org.yaml.snakeyaml.constructor.Constructor;
 
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
+import java.util.Set;
 
 /**
  * # key = demo-consumer.migration
@@ -38,14 +42,25 @@ import java.util.Map;
  * step: FORCE_APPLICATION
  */
 public class MigrationRule {
+    public static final MigrationRule INIT = new MigrationRule();
+
     private String key;
     private MigrationStep step;
     private Float threshold;
     // FIXME
     private List<String> targetIps;
     private List<InterfaceMigrationRule> interfaces;
+    private List<ApplicationMigrationRule> applications;
 
     private transient Map<String, InterfaceMigrationRule> interfaceRules;
+    private transient Map<String, ApplicationMigrationRule> applicationRules;
+
+    public MigrationRule() {
+    }
+
+    public MigrationRule(String key) {
+        this.key = key;
+    }
 
     public String getKey() {
         return key;
@@ -55,13 +70,24 @@ public class MigrationRule {
         this.key = key;
     }
 
-    public MigrationStep getStep(String serviceKey) {
+    public MigrationStep getStep(String serviceKey, Set<String> apps) {
         if (interfaceRules != null) {
             InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
             if (rule != null) {
                 return rule.getStep() == null ? step : rule.getStep();
             }
         }
+
+        if (apps != null) {
+            for (String app : apps) {
+                if (applicationRules != null) {
+                    ApplicationMigrationRule rule = applicationRules.get(app);
+                    if (rule != null) {
+                        return rule.getStep() == null ? step : rule.getStep();
+                    }
+                }
+            }
+        }
         return step;
     }
 
@@ -72,17 +98,35 @@ public class MigrationRule {
         return interfaceRules.get(serviceKey);
     }
 
+    public ApplicationMigrationRule getApplicationRule(String app) {
+        if (applicationRules == null) {
+            return null;
+        }
+        return applicationRules.get(app);
+    }
+
     public MigrationStep getStep() {
         return step;
     }
 
-    public Float getThreshold(String serviceKey) {
+    public Float getThreshold(String serviceKey, Set<String> apps) {
         if (interfaceRules != null) {
             InterfaceMigrationRule rule = interfaceRules.get(serviceKey);
             if (rule != null) {
                 return rule.getThreshold() == null ? threshold : rule.getThreshold();
             }
         }
+
+        if (apps != null) {
+            for (String app : apps) {
+                if (applicationRules != null) {
+                    ApplicationMigrationRule rule = applicationRules.get(app);
+                    if (rule != null) {
+                        return rule.getThreshold() == null ? threshold : rule.getThreshold();
+                    }
+                }
+            }
+        }
         return threshold;
     }
 
@@ -120,9 +164,83 @@ public class MigrationRule {
         }
     }
 
+    public List<ApplicationMigrationRule> getApplications() {
+        return applications;
+    }
+
+    public void setApplications(List<ApplicationMigrationRule> applications) {
+        this.applications = applications;
+        if (applications != null) {
+            this.applicationRules = new HashMap<>();
+            applications.forEach(rule -> {
+                applicationRules.put(rule.getName(), rule);
+            });
+        }
+    }
+
+    public boolean removeApplicationRule(String providerApp) {
+        if (CollectionUtils.isNotEmpty(this.applications)) {
+            boolean removed = this.applications.removeIf(applicationMigrationRule -> applicationMigrationRule.getName().equals(providerApp));
+            this.applicationRules.remove(providerApp);
+            return removed;
+        }
+        return false;
+    }
+
+    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 boolean addInterfaceRule(String providerApp, String serviceKey, MigrationStep step, Float threshold) {
+        if (getInterfaceRule(serviceKey) != null) {
+            return false;
+        }
+
+        if (this.interfaces == null) {
+            this.interfaces = new ArrayList<>();
+        }
+        InterfaceMigrationRule interfaceMigrationRule = new InterfaceMigrationRule(providerApp, serviceKey, step, threshold);
+        this.interfaces.add(interfaceMigrationRule);
+
+        if (interfaceRules == null) {
+            this.interfaceRules = new HashMap<>();
+        }
+        this.interfaceRules.put(serviceKey, interfaceMigrationRule);
+        return true;
+    }
+
+    public boolean addApplicationRule(String providerApp, MigrationStep step, Float threshold) {
+        if (getApplicationRule(providerApp) != null) {
+            return false;
+        }
+
+        if (this.applications == null) {
+            this.applications = new ArrayList<>();
+        }
+        ApplicationMigrationRule applicationMigrationRule = new ApplicationMigrationRule(providerApp, step, threshold);
+        this.applications.add(applicationMigrationRule);
+
+        if (applicationRules == null) {
+            this.applicationRules = new HashMap<>();
+        }
+        this.applicationRules.put(providerApp, applicationMigrationRule);
+        return true;
+    }
+
     public static MigrationRule parse(String rawRule) {
         Constructor constructor = new Constructor(MigrationRule.class);
         Yaml yaml = new Yaml(constructor);
         return yaml.load(rawRule);
     }
+
+    public static String toYaml(MigrationRule rule) {
+        Constructor constructor = new Constructor(MigrationRule.class);
+        Yaml yaml = new Yaml(constructor);
+        return yaml.dump(rule);
+    }
 }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/DynamicDirectory.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/DynamicDirectory.java
index 61d33be..997f41a 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/DynamicDirectory.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/DynamicDirectory.java
@@ -177,6 +177,10 @@ public abstract class DynamicDirectory<T> extends AbstractDirectory<T> implement
         return this.overrideDirectoryUrl;
     }
 
+    public URL getOriginalConsumerUrl() {
+        return this.consumerUrl;
+    }
+
     public URL getRegisteredConsumerUrl() {
         return registeredConsumerUrl;
     }
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocol.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocol.java
index 95448c0..284304d 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocol.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocol.java
@@ -50,6 +50,7 @@ import org.apache.dubbo.rpc.cluster.support.MergeableCluster;
 import org.apache.dubbo.rpc.model.ApplicationModel;
 import org.apache.dubbo.rpc.model.ProviderModel;
 import org.apache.dubbo.rpc.protocol.InvokerWrapper;
+import org.apache.dubbo.rpc.support.ProtocolUtils;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -106,6 +107,7 @@ import static org.apache.dubbo.remoting.Constants.CONNECTIONS_KEY;
 import static org.apache.dubbo.remoting.Constants.EXCHANGER_KEY;
 import static org.apache.dubbo.remoting.Constants.SERIALIZATION_KEY;
 import static org.apache.dubbo.rpc.Constants.DEPRECATED_KEY;
+import static org.apache.dubbo.rpc.Constants.GENERIC_KEY;
 import static org.apache.dubbo.rpc.Constants.INTERFACES;
 import static org.apache.dubbo.rpc.Constants.MOCK_KEY;
 import static org.apache.dubbo.rpc.Constants.TOKEN_KEY;
@@ -450,24 +452,28 @@ public class RegistryProtocol implements Protocol {
     }
 
     protected <T> Invoker<T> doRefer(Cluster cluster, Registry registry, Class<T> type, URL url, Map<String, String> parameters) {
-        URL consumerUrl = new URL(parameters.get(PROTOCOL_KEY) == null ? DUBBO : parameters.get(PROTOCOL_KEY), parameters.get(REGISTER_IP_KEY), 0, type.getName(), parameters);
+        URL consumerUrl = new URL(parameters.get(PROTOCOL_KEY) == null ? DUBBO : parameters.get(PROTOCOL_KEY), parameters.get(REGISTER_IP_KEY), 0, getPath(parameters, type), parameters);
         url = url.putAttribute(CONSUMER_URL_KEY, consumerUrl);
         ClusterInvoker<T> migrationInvoker = getMigrationInvoker(this, cluster, registry, type, url, consumerUrl);
-        return interceptInvoker(migrationInvoker, url, consumerUrl);
+        return interceptInvoker(migrationInvoker, url, consumerUrl, url);
+    }
+
+    private String getPath(Map<String, String> parameters, Class<?> type) {
+        return !ProtocolUtils.isGeneric(parameters.get(GENERIC_KEY)) ? type.getName() : parameters.get(INTERFACE_KEY);
     }
 
     protected <T> ClusterInvoker<T> getMigrationInvoker(RegistryProtocol registryProtocol, Cluster cluster, Registry registry, Class<T> type, URL url, URL consumerUrl) {
         return new ServiceDiscoveryMigrationInvoker<T>(registryProtocol, cluster, registry, type, url, consumerUrl);
     }
 
-    protected <T> Invoker<T> interceptInvoker(ClusterInvoker<T> invoker, URL url, URL consumerUrl) {
+    protected <T> Invoker<T> interceptInvoker(ClusterInvoker<T> invoker, URL url, URL consumerUrl, URL registryURL) {
         List<RegistryProtocolListener> listeners = findRegistryProtocolListeners(url);
         if (CollectionUtils.isEmpty(listeners)) {
             return invoker;
         }
 
         for (RegistryProtocolListener listener : listeners) {
-            listener.onRefer(this, invoker, consumerUrl);
+            listener.onRefer(this, invoker, consumerUrl, registryURL);
         }
         return invoker;
     }
@@ -488,7 +494,7 @@ public class RegistryProtocol implements Protocol {
         directory.setProtocol(protocol);
         // all attributes of REFER_KEY
         Map<String, String> parameters = new HashMap<String, String>(directory.getConsumerUrl().getParameters());
-        URL urlToRegistry = new URL(parameters.get(PROTOCOL_KEY) == null ? DUBBO : parameters.get(PROTOCOL_KEY), parameters.remove(REGISTER_IP_KEY), 0, type.getName(), parameters);
+        URL urlToRegistry = new URL(parameters.get(PROTOCOL_KEY) == null ? DUBBO : parameters.get(PROTOCOL_KEY), parameters.remove(REGISTER_IP_KEY), 0, getPath(parameters, type), parameters);
         URL consumerURL = directory.getConsumerUrl();
         if (directory.isShouldRegister()) {
             directory.setRegisteredConsumerUrl(urlToRegistry);
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocolListener.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocolListener.java
index c4cade5..41c57db 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocolListener.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/integration/RegistryProtocolListener.java
@@ -44,7 +44,7 @@ public interface RegistryProtocolListener {
      * @param url
      * @see RegistryProtocol#refer(Class, URL)
      */
-    void onRefer(RegistryProtocol registryProtocol, ClusterInvoker<?> invoker, URL url);
+    void onRefer(RegistryProtocol registryProtocol, ClusterInvoker<?> invoker, URL url, URL registryURL);
 
     /**
      * Notify RegistryProtocol's listeners when the protocol is destroyed
diff --git a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/support/AbstractRegistry.java b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/support/AbstractRegistry.java
index f408266..e5c64f5 100644
--- a/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/support/AbstractRegistry.java
+++ b/dubbo-registry/dubbo-registry-api/src/main/java/org/apache/dubbo/registry/support/AbstractRegistry.java
@@ -217,7 +217,7 @@ public abstract class AbstractRegistry implements Registry {
                 in = new FileInputStream(file);
                 properties.load(in);
                 if (logger.isInfoEnabled()) {
-                    logger.info("Load registry cache file " + file + ", data: " + properties);
+                    logger.info("Loaded registry cache file " + file);
                 }
             } catch (Throwable e) {
                 logger.warn("Failed to load registry cache file " + file, e);
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
index 86ff3a2..d1484c6 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/ZKTools.java
@@ -58,7 +58,8 @@ public class ZKTools {
             }
         }, executor);
 
-        testMigrationRule();
+//        testMigrationRule();
+        testAppMigrationRule();
 //        tesConditionRule();
 //        testStartupConfig();
 //        testProviderConfig();
@@ -69,10 +70,32 @@ public class ZKTools {
     }
 
     public static void testMigrationRule() {
-        String serviceStr = "---\n" +
-                "key: demo-consumer\n" +
-                "step: INTERFACE_FIRST\n" +
-                "...";
+        String serviceStr = "key: demo-consumer\n" +
+                "interfaces:\n" +
+                "  - serviceKey: org.apache.dubbo.demo.DemoService:1.0.0\n" +
+                "    threshold: 1.0\n" +
+                "    step: FORCE_APPLICATION";
+        try {
+            String servicePath = "/dubbo/config/DUBBO_SERVICEDISCOVERY_MIGRATION/demo-consumer.migration";
+            if (client.checkExists().forPath(servicePath) == null) {
+                client.create().creatingParentsIfNeeded().forPath(servicePath);
+            }
+            setData(servicePath, serviceStr);
+        } catch (Exception e) {
+            e.printStackTrace();
+        }
+    }
+
+    public static void testAppMigrationRule() {
+        String serviceStr = "key: demo-consumer\n" +
+                "applications:\n" +
+                "  - name: demo-provider\n" +
+                "    step: FORCE_APPLICATION\n" +
+                "    threshold: 0.8\n" +
+                "interfaces:\n" +
+                "  - serviceKey: org.apache.dubbo.demo.DemoService\n" +
+                "    threshold: 1.0\n" +
+                "    step: FORCE_APPLICATION";
         try {
             String servicePath = "/dubbo/config/DUBBO_SERVICEDISCOVERY_MIGRATION/demo-consumer.migration";
             if (client.checkExists().forPath(servicePath) == null) {
diff --git a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtilsTest.java b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtilsTest.java
index 4b3184b..c93de88 100644
--- a/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtilsTest.java
+++ b/dubbo-registry/dubbo-registry-api/src/test/java/org/apache/dubbo/registry/client/metadata/ServiceInstanceMetadataUtilsTest.java
@@ -38,13 +38,13 @@ public class ServiceInstanceMetadataUtilsTest {
     private static URL url = URL.valueOf("dubbo://192.168.0.102:20880/org.apache.dubbo.metadata.MetadataService?&anyhost=true&application=spring-cloud-alibaba-dubbo-provider&bind.ip=192.168.0.102&bind.port=20880&default.deprecated=false&default.dynamic=false&default.register=true&deprecated=false&dubbo=2.0.2&dynamic=false&generic=false&group=spring-cloud-alibaba-dubbo-provider&interface=org.apache.dubbo.metadata.MetadataService&methods=getAllServiceKeys,getServiceRestMetadata,getExported [...]
     private static URL url2 = URL.valueOf("rest://192.168.0.102:20880/org.apache.dubbo.metadata.MetadataService?&anyhost=true&application=spring-cloud-alibaba-dubbo-provider&bind.ip=192.168.0.102&bind.port=20880&default.deprecated=false&default.dynamic=false&default.register=true&deprecated=false&dubbo=2.0.2&dynamic=false&generic=false&group=spring-cloud-alibaba-dubbo-provider&interface=org.apache.dubbo.metadata.MetadataService&methods=getAllServiceKeys,getServiceRestMetadata,getExported [...]
 
-    private static final String VALUE = "{\"rest\":{\"version\":\"1.0.0\",\"dubbo\":\"2.0.2\",\"release\":\"2.7.1\",\"port\":\"20880\"},\"dubbo\":{\"version\":\"1.0.0\",\"dubbo\":\"2.0.2\",\"release\":\"2.7.1\",\"port\":\"20880\"}}";
+    private static final String VALUE = "{\"version\":\"1.0.0\",\"dubbo\":\"2.0.2\",\"release\":\"2.7.1\",\"port\":\"20880\"}";
     @Test
     public void testMetadataServiceURLParameters() {
 
         List<URL> urls = Arrays.asList(url, url2);
 
-        String parameter = ServiceInstanceMetadataUtils.getMetadataServiceParameter(urls);
+        String parameter = ServiceInstanceMetadataUtils.getMetadataServiceParameter(url);
 
         JSONObject jsonObject = JSON.parseObject(parameter);
 
diff --git a/dubbo-registry/dubbo-registry-nacos/src/main/java/org/apache/dubbo/registry/nacos/NacosServiceDiscovery.java b/dubbo-registry/dubbo-registry-nacos/src/main/java/org/apache/dubbo/registry/nacos/NacosServiceDiscovery.java
index 0a47aa2..10a0714 100644
--- a/dubbo-registry/dubbo-registry-nacos/src/main/java/org/apache/dubbo/registry/nacos/NacosServiceDiscovery.java
+++ b/dubbo-registry/dubbo-registry-nacos/src/main/java/org/apache/dubbo/registry/nacos/NacosServiceDiscovery.java
@@ -81,10 +81,14 @@ public class NacosServiceDiscovery extends AbstractServiceDiscovery {
 
     @Override
     public void update(ServiceInstance serviceInstance) throws RuntimeException {
-        this.serviceInstance = serviceInstance;
         // TODO: Nacos should support
-        unregister(serviceInstance);
-        register(serviceInstance);
+        if (this.serviceInstance == null) {
+            register(serviceInstance);
+        } else {
+            unregister(serviceInstance);
+            register(serviceInstance);
+            this.serviceInstance = serviceInstance;
+        }
     }
 
     @Override
diff --git a/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscovery.java b/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscovery.java
index 6e47bb0..d9f084e 100644
--- a/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscovery.java
+++ b/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscovery.java
@@ -26,6 +26,7 @@ import org.apache.dubbo.common.utils.Page;
 import org.apache.dubbo.registry.client.ServiceDiscovery;
 import org.apache.dubbo.registry.client.ServiceInstance;
 import org.apache.dubbo.registry.client.event.listener.ServiceInstancesChangedListener;
+import org.apache.dubbo.rpc.RpcException;
 
 import org.apache.curator.framework.CuratorFramework;
 import org.apache.curator.framework.api.CuratorWatcher;
@@ -45,6 +46,7 @@ import static org.apache.dubbo.registry.zookeeper.util.CuratorFrameworkParams.RO
 import static org.apache.dubbo.registry.zookeeper.util.CuratorFrameworkUtils.build;
 import static org.apache.dubbo.registry.zookeeper.util.CuratorFrameworkUtils.buildCuratorFramework;
 import static org.apache.dubbo.registry.zookeeper.util.CuratorFrameworkUtils.buildServiceDiscovery;
+import static org.apache.dubbo.rpc.RpcException.REGISTRY_EXCEPTION;
 
 /**
  * Zookeeper {@link ServiceDiscovery} implementation based on
@@ -94,17 +96,20 @@ public class ZookeeperServiceDiscovery implements ServiceDiscovery {
 
     public void register(ServiceInstance serviceInstance) throws RuntimeException {
         this.serviceInstance = serviceInstance;
-        doInServiceRegistry(serviceDiscovery -> {
+        try {
             serviceDiscovery.registerService(build(serviceInstance));
-        });
+        } catch (Exception e) {
+            throw new RpcException(REGISTRY_EXCEPTION, "Failed register instance " + serviceInstance.toString(), e);
+        }
     }
 
     public void update(ServiceInstance serviceInstance) throws RuntimeException {
-        this.serviceInstance = serviceInstance;
-        if (isInstanceUpdated(serviceInstance)) {
-            doInServiceRegistry(serviceDiscovery -> {
-                serviceDiscovery.updateService(build(serviceInstance));
-            });
+        if (this.serviceInstance == null) {
+            this.register(serviceInstance);
+        } else if (isInstanceUpdated(serviceInstance)) {
+            this.unregister(this.serviceInstance);
+            this.register(serviceInstance);
+            this.serviceInstance = serviceInstance;
         }
     }
 
@@ -189,10 +194,11 @@ public class ZookeeperServiceDiscovery implements ServiceDiscovery {
     protected void registerServiceWatcher(String serviceName, ServiceInstancesChangedListener listener) {
         String path = buildServicePath(serviceName);
         try {
-            curatorFramework.create().forPath(path);
+            curatorFramework.create().creatingParentsIfNeeded().forPath(path);
         } catch (KeeperException.NodeExistsException e) {
             // ignored
             if (logger.isDebugEnabled()) {
+
                 logger.debug(e);
             }
         } catch (Exception e) {
diff --git a/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscoveryChangeWatcher.java b/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscoveryChangeWatcher.java
index fef3fef..e34d3e4 100644
--- a/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscoveryChangeWatcher.java
+++ b/dubbo-registry/dubbo-registry-zookeeper/src/main/java/org/apache/dubbo/registry/zookeeper/ZookeeperServiceDiscoveryChangeWatcher.java
@@ -43,7 +43,7 @@ public class ZookeeperServiceDiscoveryChangeWatcher implements CuratorWatcher {
 
     private final RegistryNotifier notifier;
 
-    private boolean keepWatching;
+    private boolean keepWatching = true;
 
     private final String serviceName;
 
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
index a95165a..3228deb 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/RpcException.java
@@ -37,6 +37,7 @@ public /**final**/ class RpcException extends RuntimeException {
     public static final int NO_INVOKER_AVAILABLE_AFTER_FILTER = 6;
     public static final int LIMIT_EXCEEDED_EXCEPTION = 7;
     public static final int TIMEOUT_TERMINATE = 8;
+    public static final int REGISTRY_EXCEPTION = 9;
     private static final long serialVersionUID = 7815426752583648734L;
     /**
      * RpcException cannot be extended, use error code for exception type to keep compatibility
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ActiveLimitFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ActiveLimitFilter.java
index 81f8c9a..e1d8af7 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ActiveLimitFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ActiveLimitFilter.java
@@ -1,122 +1,122 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.RpcStatus;
-
-import static org.apache.dubbo.common.constants.CommonConstants.CONSUMER;
-import static org.apache.dubbo.common.constants.CommonConstants.TIMEOUT_KEY;
-import static org.apache.dubbo.rpc.Constants.ACTIVES_KEY;
-
-/**
- * ActiveLimitFilter restrict the concurrent client invocation for a service or service's method from client side.
- * To use active limit filter, configured url with <b>actives</b> and provide valid >0 integer value.
- * <pre>
- *     e.g. <dubbo:reference id="demoService" check="false" interface="org.apache.dubbo.demo.DemoService" "actives"="2"/>
- *      In the above example maximum 2 concurrent invocation is allowed.
- *      If there are more than configured (in this example 2) is trying to invoke remote method, then rest of invocation
- *      will wait for configured timeout(default is 0 second) before invocation gets kill by dubbo.
- * </pre>
- *
- * @see Filter
- */
-@Activate(group = CONSUMER, value = ACTIVES_KEY)
-public class ActiveLimitFilter implements Filter, Filter.Listener {
-
-    private static final String ACTIVELIMIT_FILTER_START_TIME = "activelimit_filter_start_time";
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        URL url = invoker.getUrl();
-        String methodName = invocation.getMethodName();
-        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
-        final RpcStatus rpcStatus = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
-        if (!RpcStatus.beginCount(url, methodName, max)) {
-            long timeout = invoker.getUrl().getMethodParameter(invocation.getMethodName(), TIMEOUT_KEY, 0);
-            long start = System.currentTimeMillis();
-            long remain = timeout;
-            synchronized (rpcStatus) {
-                while (!RpcStatus.beginCount(url, methodName, max)) {
-                    try {
-                        rpcStatus.wait(remain);
-                    } catch (InterruptedException e) {
-                        // ignore
-                    }
-                    long elapsed = System.currentTimeMillis() - start;
-                    remain = timeout - elapsed;
-                    if (remain <= 0) {
-                        throw new RpcException(RpcException.LIMIT_EXCEEDED_EXCEPTION,
-                                "Waiting concurrent invoke timeout in client-side for service:  " +
-                                        invoker.getInterface().getName() + ", method: " + invocation.getMethodName() +
-                                        ", elapsed: " + elapsed + ", timeout: " + timeout + ". concurrent invokes: " +
-                                        rpcStatus.getActive() + ". max concurrent invoke limit: " + max);
-                    }
-                }
-            }
-        }
-
-        invocation.put(ACTIVELIMIT_FILTER_START_TIME, System.currentTimeMillis());
-
-        return invoker.invoke(invocation);
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        String methodName = invocation.getMethodName();
-        URL url = invoker.getUrl();
-        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
-
-        RpcStatus.endCount(url, methodName, getElapsed(invocation), true);
-        notifyFinish(RpcStatus.getStatus(url, methodName), max);
-    }
-
-    @Override
-    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
-        String methodName = invocation.getMethodName();
-        URL url = invoker.getUrl();
-        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
-
-        if (t instanceof RpcException) {
-            RpcException rpcException = (RpcException) t;
-            if (rpcException.isLimitExceed()) {
-                return;
-            }
-        }
-        RpcStatus.endCount(url, methodName, getElapsed(invocation), false);
-        notifyFinish(RpcStatus.getStatus(url, methodName), max);
-    }
-
-    private long getElapsed(Invocation invocation) {
-        Object beginTime = invocation.get(ACTIVELIMIT_FILTER_START_TIME);
-        return beginTime != null ? System.currentTimeMillis() - (Long) beginTime : 0;
-    }
-
-    private void notifyFinish(final RpcStatus rpcStatus, int max) {
-        if (max > 0) {
-            synchronized (rpcStatus) {
-                rpcStatus.notifyAll();
-            }
-        }
-    }
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.RpcStatus;
+
+import static org.apache.dubbo.common.constants.CommonConstants.CONSUMER;
+import static org.apache.dubbo.common.constants.CommonConstants.TIMEOUT_KEY;
+import static org.apache.dubbo.rpc.Constants.ACTIVES_KEY;
+
+/**
+ * ActiveLimitFilter restrict the concurrent client invocation for a service or service's method from client side.
+ * To use active limit filter, configured url with <b>actives</b> and provide valid >0 integer value.
+ * <pre>
+ *     e.g. <dubbo:reference id="demoService" check="false" interface="org.apache.dubbo.demo.DemoService" "actives"="2"/>
+ *      In the above example maximum 2 concurrent invocation is allowed.
+ *      If there are more than configured (in this example 2) is trying to invoke remote method, then rest of invocation
+ *      will wait for configured timeout(default is 0 second) before invocation gets kill by dubbo.
+ * </pre>
+ *
+ * @see Filter
+ */
+@Activate(group = CONSUMER, value = ACTIVES_KEY)
+public class ActiveLimitFilter implements Filter, Filter.Listener {
+
+    private static final String ACTIVELIMIT_FILTER_START_TIME = "activelimit_filter_start_time";
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        URL url = invoker.getUrl();
+        String methodName = invocation.getMethodName();
+        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
+        final RpcStatus rpcStatus = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
+        if (!RpcStatus.beginCount(url, methodName, max)) {
+            long timeout = invoker.getUrl().getMethodParameter(invocation.getMethodName(), TIMEOUT_KEY, 0);
+            long start = System.currentTimeMillis();
+            long remain = timeout;
+            synchronized (rpcStatus) {
+                while (!RpcStatus.beginCount(url, methodName, max)) {
+                    try {
+                        rpcStatus.wait(remain);
+                    } catch (InterruptedException e) {
+                        // ignore
+                    }
+                    long elapsed = System.currentTimeMillis() - start;
+                    remain = timeout - elapsed;
+                    if (remain <= 0) {
+                        throw new RpcException(RpcException.LIMIT_EXCEEDED_EXCEPTION,
+                                "Waiting concurrent invoke timeout in client-side for service:  " +
+                                        invoker.getInterface().getName() + ", method: " + invocation.getMethodName() +
+                                        ", elapsed: " + elapsed + ", timeout: " + timeout + ". concurrent invokes: " +
+                                        rpcStatus.getActive() + ". max concurrent invoke limit: " + max);
+                    }
+                }
+            }
+        }
+
+        invocation.put(ACTIVELIMIT_FILTER_START_TIME, System.currentTimeMillis());
+
+        return invoker.invoke(invocation);
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        String methodName = invocation.getMethodName();
+        URL url = invoker.getUrl();
+        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
+
+        RpcStatus.endCount(url, methodName, getElapsed(invocation), true);
+        notifyFinish(RpcStatus.getStatus(url, methodName), max);
+    }
+
+    @Override
+    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
+        String methodName = invocation.getMethodName();
+        URL url = invoker.getUrl();
+        int max = invoker.getUrl().getMethodParameter(methodName, ACTIVES_KEY, 0);
+
+        if (t instanceof RpcException) {
+            RpcException rpcException = (RpcException) t;
+            if (rpcException.isLimitExceed()) {
+                return;
+            }
+        }
+        RpcStatus.endCount(url, methodName, getElapsed(invocation), false);
+        notifyFinish(RpcStatus.getStatus(url, methodName), max);
+    }
+
+    private long getElapsed(Invocation invocation) {
+        Object beginTime = invocation.get(ACTIVELIMIT_FILTER_START_TIME);
+        return beginTime != null ? System.currentTimeMillis() - (Long) beginTime : 0;
+    }
+
+    private void notifyFinish(final RpcStatus rpcStatus, int max) {
+        if (max > 0) {
+            synchronized (rpcStatus) {
+                rpcStatus.notifyAll();
+            }
+        }
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ClassLoaderFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ClassLoaderFilter.java
index 62ba785..c71e2d1 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ClassLoaderFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ClassLoaderFilter.java
@@ -1,44 +1,44 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-
-/**
- * Set the current execution thread class loader to service interface's class loader.
- */
-@Activate(group = CommonConstants.PROVIDER, order = -30000)
-public class ClassLoaderFilter implements Filter {
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        ClassLoader ocl = Thread.currentThread().getContextClassLoader();
-        Thread.currentThread().setContextClassLoader(invoker.getInterface().getClassLoader());
-        try {
-            return invoker.invoke(invocation);
-        } finally {
-            Thread.currentThread().setContextClassLoader(ocl);
-        }
-    }
-
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+
+/**
+ * Set the current execution thread class loader to service interface's class loader.
+ */
+@Activate(group = CommonConstants.PROVIDER, order = -30000)
+public class ClassLoaderFilter implements Filter {
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        ClassLoader ocl = Thread.currentThread().getContextClassLoader();
+        Thread.currentThread().setContextClassLoader(invoker.getInterface().getClassLoader());
+        try {
+            return invoker.invoke(invocation);
+        } finally {
+            Thread.currentThread().setContextClassLoader(ocl);
+        }
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/CompatibleFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/CompatibleFilter.java
index ba37a86..b7b368f 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/CompatibleFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/CompatibleFilter.java
@@ -1,92 +1,92 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
-import org.apache.dubbo.common.utils.CompatibleTypeUtils;
-import org.apache.dubbo.common.utils.PojoUtils;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-
-import java.lang.reflect.Method;
-import java.lang.reflect.Type;
-
-import static org.apache.dubbo.remoting.Constants.SERIALIZATION_KEY;
-
-/**
- * CompatibleFilter make the remote method's return value compatible to invoker's version of object.
- * To make return object compatible it does
- * <pre>
- *    1)If the url contain serialization key of type <b>json</b> or <b>fastjson</b> then transform
- *    the return value to instance of {@link java.util.Map}
- *    2)If the return value is not a instance of invoked method's return type available at
- *    local jvm then POJO conversion.
- *    3)If return value is other than above return value as it is.
- * </pre>
- *
- * @see Filter
- *
- */
-public class CompatibleFilter implements Filter, Filter.Listener {
-
-    private static Logger logger = LoggerFactory.getLogger(CompatibleFilter.class);
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        return invoker.invoke(invocation);
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        if (!invocation.getMethodName().startsWith("$") && !appResponse.hasException()) {
-            Object value = appResponse.getValue();
-            if (value != null) {
-                try {
-                    Method method = invoker.getInterface().getMethod(invocation.getMethodName(), invocation.getParameterTypes());
-                    Class<?> type = method.getReturnType();
-                    Object newValue;
-                    String serialization = invoker.getUrl().getParameter(SERIALIZATION_KEY);
-                    if ("json".equals(serialization) || "fastjson".equals(serialization)) {
-                        // If the serialization key is json or fastjson
-                        Type gtype = method.getGenericReturnType();
-                        newValue = PojoUtils.realize(value, type, gtype);
-                    } else if (!type.isInstance(value)) {
-                        //if local service interface's method's return type is not instance of return value
-                        newValue = PojoUtils.isPojo(type) ? PojoUtils.realize(value, type) : CompatibleTypeUtils.compatibleTypeConvert(value, type);
-
-                    } else {
-                        newValue = value;
-                    }
-                    if (newValue != value) {
-                        appResponse.setValue(newValue);
-                    }
-                } catch (Throwable t) {
-                    logger.warn(t.getMessage(), t);
-                }
-            }
-        }
-    }
-
-    @Override
-    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
-
-    }
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.common.utils.CompatibleTypeUtils;
+import org.apache.dubbo.common.utils.PojoUtils;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+
+import static org.apache.dubbo.remoting.Constants.SERIALIZATION_KEY;
+
+/**
+ * CompatibleFilter make the remote method's return value compatible to invoker's version of object.
+ * To make return object compatible it does
+ * <pre>
+ *    1)If the url contain serialization key of type <b>json</b> or <b>fastjson</b> then transform
+ *    the return value to instance of {@link java.util.Map}
+ *    2)If the return value is not a instance of invoked method's return type available at
+ *    local jvm then POJO conversion.
+ *    3)If return value is other than above return value as it is.
+ * </pre>
+ *
+ * @see Filter
+ *
+ */
+public class CompatibleFilter implements Filter, Filter.Listener {
+
+    private static Logger logger = LoggerFactory.getLogger(CompatibleFilter.class);
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        return invoker.invoke(invocation);
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        if (!invocation.getMethodName().startsWith("$") && !appResponse.hasException()) {
+            Object value = appResponse.getValue();
+            if (value != null) {
+                try {
+                    Method method = invoker.getInterface().getMethod(invocation.getMethodName(), invocation.getParameterTypes());
+                    Class<?> type = method.getReturnType();
+                    Object newValue;
+                    String serialization = invoker.getUrl().getParameter(SERIALIZATION_KEY);
+                    if ("json".equals(serialization) || "fastjson".equals(serialization)) {
+                        // If the serialization key is json or fastjson
+                        Type gtype = method.getGenericReturnType();
+                        newValue = PojoUtils.realize(value, type, gtype);
+                    } else if (!type.isInstance(value)) {
+                        //if local service interface's method's return type is not instance of return value
+                        newValue = PojoUtils.isPojo(type) ? PojoUtils.realize(value, type) : CompatibleTypeUtils.compatibleTypeConvert(value, type);
+
+                    } else {
+                        newValue = value;
+                    }
+                    if (newValue != value) {
+                        appResponse.setValue(newValue);
+                    }
+                } catch (Throwable t) {
+                    logger.warn(t.getMessage(), t);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
+
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/DeprecatedFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/DeprecatedFilter.java
index 112951a..c4f5695 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/DeprecatedFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/DeprecatedFilter.java
@@ -1,78 +1,78 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
-import org.apache.dubbo.common.utils.ConcurrentHashSet;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-
-import java.util.Set;
-
-import static org.apache.dubbo.rpc.Constants.DEPRECATED_KEY;
-
-/**
- * DeprecatedFilter logs error message if a invoked method has been marked as deprecated. To check whether a method
- * is deprecated or not it looks for <b>deprecated</b> attribute value and consider it is deprecated it value is <b>true</b>
- *
- * @see Filter
- */
-@Activate(group = CommonConstants.CONSUMER, value = DEPRECATED_KEY)
-public class DeprecatedFilter implements Filter {
-
-    private static final Logger LOGGER = LoggerFactory.getLogger(DeprecatedFilter.class);
-
-    private static final Set<String> LOGGED = new ConcurrentHashSet<String>();
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        String key = invoker.getInterface().getName() + "." + invocation.getMethodName();
-        if (!LOGGED.contains(key)) {
-            LOGGED.add(key);
-            if (invoker.getUrl().getMethodParameter(invocation.getMethodName(), DEPRECATED_KEY, false)) {
-                LOGGER.error("The service method " + invoker.getInterface().getName() + "." + getMethodSignature(invocation) + " is DEPRECATED! Declare from " + invoker.getUrl());
-            }
-        }
-        return invoker.invoke(invocation);
-    }
-
-    private String getMethodSignature(Invocation invocation) {
-        StringBuilder buf = new StringBuilder(invocation.getMethodName());
-        buf.append("(");
-        Class<?>[] types = invocation.getParameterTypes();
-        if (types != null && types.length > 0) {
-            boolean first = true;
-            for (Class<?> type : types) {
-                if (first) {
-                    first = false;
-                } else {
-                    buf.append(", ");
-                }
-                buf.append(type.getSimpleName());
-            }
-        }
-        buf.append(")");
-        return buf.toString();
-    }
-
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.common.utils.ConcurrentHashSet;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+
+import java.util.Set;
+
+import static org.apache.dubbo.rpc.Constants.DEPRECATED_KEY;
+
+/**
+ * DeprecatedFilter logs error message if a invoked method has been marked as deprecated. To check whether a method
+ * is deprecated or not it looks for <b>deprecated</b> attribute value and consider it is deprecated it value is <b>true</b>
+ *
+ * @see Filter
+ */
+@Activate(group = CommonConstants.CONSUMER, value = DEPRECATED_KEY)
+public class DeprecatedFilter implements Filter {
+
+    private static final Logger LOGGER = LoggerFactory.getLogger(DeprecatedFilter.class);
+
+    private static final Set<String> LOGGED = new ConcurrentHashSet<String>();
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        String key = invoker.getInterface().getName() + "." + invocation.getMethodName();
+        if (!LOGGED.contains(key)) {
+            LOGGED.add(key);
+            if (invoker.getUrl().getMethodParameter(invocation.getMethodName(), DEPRECATED_KEY, false)) {
+                LOGGER.error("The service method " + invoker.getInterface().getName() + "." + getMethodSignature(invocation) + " is DEPRECATED! Declare from " + invoker.getUrl());
+            }
+        }
+        return invoker.invoke(invocation);
+    }
+
+    private String getMethodSignature(Invocation invocation) {
+        StringBuilder buf = new StringBuilder(invocation.getMethodName());
+        buf.append("(");
+        Class<?>[] types = invocation.getParameterTypes();
+        if (types != null && types.length > 0) {
+            boolean first = true;
+            for (Class<?> type : types) {
+                if (first) {
+                    first = false;
+                } else {
+                    buf.append(", ");
+                }
+                buf.append(type.getSimpleName());
+            }
+        }
+        buf.append(")");
+        return buf.toString();
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/EchoFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/EchoFilter.java
index d51e6d4..f3dbe01 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/EchoFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/EchoFilter.java
@@ -1,44 +1,44 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.rpc.AsyncRpcResult;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-
-import static org.apache.dubbo.rpc.Constants.$ECHO;
-
-/**
- * Dubbo provided default Echo echo service, which is available for all dubbo provider service interface.
- */
-@Activate(group = CommonConstants.PROVIDER, order = -110000)
-public class EchoFilter implements Filter {
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation inv) throws RpcException {
-        if (inv.getMethodName().equals($ECHO) && inv.getArguments() != null && inv.getArguments().length == 1) {
-            return AsyncRpcResult.newDefaultAsyncResult(inv.getArguments()[0], inv);
-        }
-        return invoker.invoke(inv);
-    }
-
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.rpc.AsyncRpcResult;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+
+import static org.apache.dubbo.rpc.Constants.$ECHO;
+
+/**
+ * Dubbo provided default Echo echo service, which is available for all dubbo provider service interface.
+ */
+@Activate(group = CommonConstants.PROVIDER, order = -110000)
+public class EchoFilter implements Filter {
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation inv) throws RpcException {
+        if (inv.getMethodName().equals($ECHO) && inv.getArguments() != null && inv.getArguments().length == 1) {
+            return AsyncRpcResult.newDefaultAsyncResult(inv.getArguments()[0], inv);
+        }
+        return invoker.invoke(inv);
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExceptionFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExceptionFilter.java
index f449644..ca98dcc 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExceptionFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExceptionFilter.java
@@ -1,115 +1,115 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
-import org.apache.dubbo.common.utils.ReflectUtils;
-import org.apache.dubbo.common.utils.StringUtils;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcContext;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.service.GenericService;
-
-import java.lang.reflect.Method;
-
-
-/**
- * ExceptionInvokerFilter
- * <p>
- * Functions:
- * <ol>
- * <li>unexpected exception will be logged in ERROR level on provider side. Unexpected exception are unchecked
- * exception not declared on the interface</li>
- * <li>Wrap the exception not introduced in API package into RuntimeException. Framework will serialize the outer exception but stringnize its cause in order to avoid of possible serialization problem on client side</li>
- * </ol>
- */
-@Activate(group = CommonConstants.PROVIDER)
-public class ExceptionFilter implements Filter, Filter.Listener {
-    private Logger logger = LoggerFactory.getLogger(ExceptionFilter.class);
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        return invoker.invoke(invocation);
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        if (appResponse.hasException() && GenericService.class != invoker.getInterface()) {
-            try {
-                Throwable exception = appResponse.getException();
-
-                // directly throw if it's checked exception
-                if (!(exception instanceof RuntimeException) && (exception instanceof Exception)) {
-                    return;
-                }
-                // directly throw if the exception appears in the signature
-                try {
-                    Method method = invoker.getInterface().getMethod(invocation.getMethodName(), invocation.getParameterTypes());
-                    Class<?>[] exceptionClassses = method.getExceptionTypes();
-                    for (Class<?> exceptionClass : exceptionClassses) {
-                        if (exception.getClass().equals(exceptionClass)) {
-                            return;
-                        }
-                    }
-                } catch (NoSuchMethodException e) {
-                    return;
-                }
-
-                // for the exception not found in method's signature, print ERROR message in server's log.
-                logger.error("Got unchecked and undeclared exception which called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + exception.getClass().getName() + ": " + exception.getMessage(), exception);
-
-                // directly throw if exception class and interface class are in the same jar file.
-                String serviceFile = ReflectUtils.getCodeBase(invoker.getInterface());
-                String exceptionFile = ReflectUtils.getCodeBase(exception.getClass());
-                if (serviceFile == null || exceptionFile == null || serviceFile.equals(exceptionFile)) {
-                    return;
-                }
-                // directly throw if it's JDK exception
-                String className = exception.getClass().getName();
-                if (className.startsWith("java.") || className.startsWith("javax.")) {
-                    return;
-                }
-                // directly throw if it's dubbo exception
-                if (exception instanceof RpcException) {
-                    return;
-                }
-
-                // otherwise, wrap with RuntimeException and throw back to the client
-                appResponse.setException(new RuntimeException(StringUtils.toString(exception)));
-            } catch (Throwable e) {
-                logger.warn("Fail to ExceptionFilter when called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + e.getClass().getName() + ": " + e.getMessage(), e);
-            }
-        }
-    }
-
-    @Override
-    public void onError(Throwable e, Invoker<?> invoker, Invocation invocation) {
-        logger.error("Got unchecked and undeclared exception which called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + e.getClass().getName() + ": " + e.getMessage(), e);
-    }
-
-    // For test purpose
-    public void setLogger(Logger logger) {
-        this.logger = logger;
-    }
-}
-
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.common.utils.ReflectUtils;
+import org.apache.dubbo.common.utils.StringUtils;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.service.GenericService;
+
+import java.lang.reflect.Method;
+
+
+/**
+ * ExceptionInvokerFilter
+ * <p>
+ * Functions:
+ * <ol>
+ * <li>unexpected exception will be logged in ERROR level on provider side. Unexpected exception are unchecked
+ * exception not declared on the interface</li>
+ * <li>Wrap the exception not introduced in API package into RuntimeException. Framework will serialize the outer exception but stringnize its cause in order to avoid of possible serialization problem on client side</li>
+ * </ol>
+ */
+@Activate(group = CommonConstants.PROVIDER)
+public class ExceptionFilter implements Filter, Filter.Listener {
+    private Logger logger = LoggerFactory.getLogger(ExceptionFilter.class);
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        return invoker.invoke(invocation);
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        if (appResponse.hasException() && GenericService.class != invoker.getInterface()) {
+            try {
+                Throwable exception = appResponse.getException();
+
+                // directly throw if it's checked exception
+                if (!(exception instanceof RuntimeException) && (exception instanceof Exception)) {
+                    return;
+                }
+                // directly throw if the exception appears in the signature
+                try {
+                    Method method = invoker.getInterface().getMethod(invocation.getMethodName(), invocation.getParameterTypes());
+                    Class<?>[] exceptionClassses = method.getExceptionTypes();
+                    for (Class<?> exceptionClass : exceptionClassses) {
+                        if (exception.getClass().equals(exceptionClass)) {
+                            return;
+                        }
+                    }
+                } catch (NoSuchMethodException e) {
+                    return;
+                }
+
+                // for the exception not found in method's signature, print ERROR message in server's log.
+                logger.error("Got unchecked and undeclared exception which called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + exception.getClass().getName() + ": " + exception.getMessage(), exception);
+
+                // directly throw if exception class and interface class are in the same jar file.
+                String serviceFile = ReflectUtils.getCodeBase(invoker.getInterface());
+                String exceptionFile = ReflectUtils.getCodeBase(exception.getClass());
+                if (serviceFile == null || exceptionFile == null || serviceFile.equals(exceptionFile)) {
+                    return;
+                }
+                // directly throw if it's JDK exception
+                String className = exception.getClass().getName();
+                if (className.startsWith("java.") || className.startsWith("javax.")) {
+                    return;
+                }
+                // directly throw if it's dubbo exception
+                if (exception instanceof RpcException) {
+                    return;
+                }
+
+                // otherwise, wrap with RuntimeException and throw back to the client
+                appResponse.setException(new RuntimeException(StringUtils.toString(exception)));
+            } catch (Throwable e) {
+                logger.warn("Fail to ExceptionFilter when called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + e.getClass().getName() + ": " + e.getMessage(), e);
+            }
+        }
+    }
+
+    @Override
+    public void onError(Throwable e, Invoker<?> invoker, Invocation invocation) {
+        logger.error("Got unchecked and undeclared exception which called by " + RpcContext.getContext().getRemoteHost() + ". service: " + invoker.getInterface().getName() + ", method: " + invocation.getMethodName() + ", exception: " + e.getClass().getName() + ": " + e.getMessage(), e);
+    }
+
+    // For test purpose
+    public void setLogger(Logger logger) {
+        this.logger = logger;
+    }
+}
+
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExecuteLimitFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExecuteLimitFilter.java
index 40bf711..63e8caf 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExecuteLimitFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/ExecuteLimitFilter.java
@@ -1,86 +1,86 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.RpcStatus;
-
-import static org.apache.dubbo.rpc.Constants.EXECUTES_KEY;
-
-
-/**
- * The maximum parallel execution request count per method per service for the provider.If the max configured
- * <b>executes</b> is set to 10 and if invoke request where it is already 10 then it will throws exception. It
- * continue the same behaviour un till it is <10.
- */
-@Activate(group = CommonConstants.PROVIDER, value = EXECUTES_KEY)
-public class ExecuteLimitFilter implements Filter, Filter.Listener {
-
-    private static final String EXECUTE_LIMIT_FILTER_START_TIME = "execute_limit_filter_start_time";
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        URL url = invoker.getUrl();
-        String methodName = invocation.getMethodName();
-        int max = url.getMethodParameter(methodName, EXECUTES_KEY, 0);
-        if (!RpcStatus.beginCount(url, methodName, max)) {
-            throw new RpcException(RpcException.LIMIT_EXCEEDED_EXCEPTION,
-                    "Failed to invoke method " + invocation.getMethodName() + " in provider " +
-                            url + ", cause: The service using threads greater than <dubbo:service executes=\"" + max +
-                            "\" /> limited.");
-        }
-
-        invocation.put(EXECUTE_LIMIT_FILTER_START_TIME, System.currentTimeMillis());
-        try {
-            return invoker.invoke(invocation);
-        } catch (Throwable t) {
-            if (t instanceof RuntimeException) {
-                throw (RuntimeException) t;
-            } else {
-                throw new RpcException("unexpected exception when ExecuteLimitFilter", t);
-            }
-        }
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        RpcStatus.endCount(invoker.getUrl(), invocation.getMethodName(), getElapsed(invocation), true);
-    }
-
-    @Override
-    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
-        if (t instanceof RpcException) {
-            RpcException rpcException = (RpcException) t;
-            if (rpcException.isLimitExceed()) {
-                return;
-            }
-        }
-        RpcStatus.endCount(invoker.getUrl(), invocation.getMethodName(), getElapsed(invocation), false);
-    }
-
-    private long getElapsed(Invocation invocation) {
-        Object beginTime = invocation.get(EXECUTE_LIMIT_FILTER_START_TIME);
-        return beginTime != null ? System.currentTimeMillis() - (Long) beginTime : 0;
-    }
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.RpcStatus;
+
+import static org.apache.dubbo.rpc.Constants.EXECUTES_KEY;
+
+
+/**
+ * The maximum parallel execution request count per method per service for the provider.If the max configured
+ * <b>executes</b> is set to 10 and if invoke request where it is already 10 then it will throws exception. It
+ * continue the same behaviour un till it is <10.
+ */
+@Activate(group = CommonConstants.PROVIDER, value = EXECUTES_KEY)
+public class ExecuteLimitFilter implements Filter, Filter.Listener {
+
+    private static final String EXECUTE_LIMIT_FILTER_START_TIME = "execute_limit_filter_start_time";
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        URL url = invoker.getUrl();
+        String methodName = invocation.getMethodName();
+        int max = url.getMethodParameter(methodName, EXECUTES_KEY, 0);
+        if (!RpcStatus.beginCount(url, methodName, max)) {
+            throw new RpcException(RpcException.LIMIT_EXCEEDED_EXCEPTION,
+                    "Failed to invoke method " + invocation.getMethodName() + " in provider " +
+                            url + ", cause: The service using threads greater than <dubbo:service executes=\"" + max +
+                            "\" /> limited.");
+        }
+
+        invocation.put(EXECUTE_LIMIT_FILTER_START_TIME, System.currentTimeMillis());
+        try {
+            return invoker.invoke(invocation);
+        } catch (Throwable t) {
+            if (t instanceof RuntimeException) {
+                throw (RuntimeException) t;
+            } else {
+                throw new RpcException("unexpected exception when ExecuteLimitFilter", t);
+            }
+        }
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        RpcStatus.endCount(invoker.getUrl(), invocation.getMethodName(), getElapsed(invocation), true);
+    }
+
+    @Override
+    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
+        if (t instanceof RpcException) {
+            RpcException rpcException = (RpcException) t;
+            if (rpcException.isLimitExceed()) {
+                return;
+            }
+        }
+        RpcStatus.endCount(invoker.getUrl(), invocation.getMethodName(), getElapsed(invocation), false);
+    }
+
+    private long getElapsed(Invocation invocation) {
+        Object beginTime = invocation.get(EXECUTE_LIMIT_FILTER_START_TIME);
+        return beginTime != null ? System.currentTimeMillis() - (Long) beginTime : 0;
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/GenericImplFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/GenericImplFilter.java
index 20ef2ce..31f6b50 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/GenericImplFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/GenericImplFilter.java
@@ -1,229 +1,229 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.beanutil.JavaBeanAccessor;
-import org.apache.dubbo.common.beanutil.JavaBeanDescriptor;
-import org.apache.dubbo.common.beanutil.JavaBeanSerializeUtil;
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
-import org.apache.dubbo.common.utils.PojoUtils;
-import org.apache.dubbo.common.utils.ReflectUtils;
-import org.apache.dubbo.rpc.Constants;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.RpcInvocation;
-import org.apache.dubbo.rpc.service.GenericService;
-import org.apache.dubbo.rpc.support.ProtocolUtils;
-import org.apache.dubbo.rpc.support.RpcUtils;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
-import java.lang.reflect.Type;
-
-import static org.apache.dubbo.common.constants.CommonConstants.$INVOKE;
-import static org.apache.dubbo.common.constants.CommonConstants.$INVOKE_ASYNC;
-import static org.apache.dubbo.common.constants.CommonConstants.GENERIC_PARAMETER_DESC;
-import static org.apache.dubbo.rpc.Constants.GENERIC_KEY;
-
-/**
- * GenericImplInvokerFilter
- */
-@Activate(group = CommonConstants.CONSUMER, value = GENERIC_KEY, order = 20000)
-public class GenericImplFilter implements Filter, Filter.Listener {
-
-    private static final Logger logger = LoggerFactory.getLogger(GenericImplFilter.class);
-
-    private static final Class<?>[] GENERIC_PARAMETER_TYPES = new Class<?>[]{String.class, String[].class, Object[].class};
-
-    private static final String GENERIC_IMPL_MARKER = "GENERIC_IMPL";
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        String generic = invoker.getUrl().getParameter(GENERIC_KEY);
-        // calling a generic impl service
-        if (isCallingGenericImpl(generic, invocation)) {
-            RpcInvocation invocation2 = new RpcInvocation(invocation);
-
-            /**
-             * Mark this invocation as a generic impl call, this value will be removed automatically before passing on the wire.
-             * See {@link RpcUtils#sieveUnnecessaryAttachments(Invocation)}
-             */
-            invocation2.put(GENERIC_IMPL_MARKER, true);
-
-            String methodName = invocation2.getMethodName();
-            Class<?>[] parameterTypes = invocation2.getParameterTypes();
-            Object[] arguments = invocation2.getArguments();
-
-            String[] types = new String[parameterTypes.length];
-            for (int i = 0; i < parameterTypes.length; i++) {
-                types[i] = ReflectUtils.getName(parameterTypes[i]);
-            }
-
-            Object[] args;
-            if (ProtocolUtils.isBeanGenericSerialization(generic)) {
-                args = new Object[arguments.length];
-                for (int i = 0; i < arguments.length; i++) {
-                    args[i] = JavaBeanSerializeUtil.serialize(arguments[i], JavaBeanAccessor.METHOD);
-                }
-            } else {
-                args = PojoUtils.generalize(arguments);
-            }
-
-            if (RpcUtils.isReturnTypeFuture(invocation)) {
-                invocation2.setMethodName($INVOKE_ASYNC);
-            } else {
-                invocation2.setMethodName($INVOKE);
-            }
-            invocation2.setParameterTypes(GENERIC_PARAMETER_TYPES);
-            invocation2.setParameterTypesDesc(GENERIC_PARAMETER_DESC);
-            invocation2.setArguments(new Object[]{methodName, types, args});
-            return invoker.invoke(invocation2);
-        }
-        // making a generic call to a normal service
-        else if (isMakingGenericCall(generic, invocation)) {
-
-            Object[] args = (Object[]) invocation.getArguments()[2];
-            if (ProtocolUtils.isJavaGenericSerialization(generic)) {
-
-                for (Object arg : args) {
-                    if (!(byte[].class == arg.getClass())) {
-                        error(generic, byte[].class.getName(), arg.getClass().getName());
-                    }
-                }
-            } else if (ProtocolUtils.isBeanGenericSerialization(generic)) {
-                for (Object arg : args) {
-                    if (!(arg instanceof JavaBeanDescriptor)) {
-                        error(generic, JavaBeanDescriptor.class.getName(), arg.getClass().getName());
-                    }
-                }
-            }
-
-            invocation.setAttachment(
-                    GENERIC_KEY, invoker.getUrl().getParameter(GENERIC_KEY));
-        }
-        return invoker.invoke(invocation);
-    }
-
-    private void error(String generic, String expected, String actual) throws RpcException {
-        throw new RpcException("Generic serialization [" + generic + "] only support message type " + expected + " and your message type is " + actual);
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        String generic = invoker.getUrl().getParameter(GENERIC_KEY);
-        String methodName = invocation.getMethodName();
-        Class<?>[] parameterTypes = invocation.getParameterTypes();
-        Object genericImplMarker = invocation.get(GENERIC_IMPL_MARKER);
-        if (genericImplMarker != null && (boolean) invocation.get(GENERIC_IMPL_MARKER)) {
-            if (!appResponse.hasException()) {
-                Object value = appResponse.getValue();
-                try {
-                    Class<?> invokerInterface = invoker.getInterface();
-                    if (!$INVOKE.equals(methodName) && !$INVOKE_ASYNC.equals(methodName)
-                            && invokerInterface.isAssignableFrom(GenericService.class)) {
-                        try {
-                            // find the real interface from url
-                            String realInterface = invoker.getUrl().getParameter(Constants.INTERFACE);
-                            invokerInterface = ReflectUtils.forName(realInterface);
-                        } catch (Throwable e) {
-                            // ignore
-                        }
-                    }
-
-                    Method method = invokerInterface.getMethod(methodName, parameterTypes);
-                    if (ProtocolUtils.isBeanGenericSerialization(generic)) {
-                        if (value == null) {
-                            appResponse.setValue(value);
-                        } else if (value instanceof JavaBeanDescriptor) {
-                            appResponse.setValue(JavaBeanSerializeUtil.deserialize((JavaBeanDescriptor) value));
-                        } else {
-                            throw new RpcException("The type of result value is " + value.getClass().getName() + " other than " + JavaBeanDescriptor.class.getName() + ", and the result is " + value);
-                        }
-                    } else {
-                        Type[] types = ReflectUtils.getReturnTypes(method);
-                        appResponse.setValue(PojoUtils.realize(value, (Class<?>) types[0], types[1]));
-                    }
-                } catch (NoSuchMethodException e) {
-                    throw new RpcException(e.getMessage(), e);
-                }
-            } else if (appResponse.getException() instanceof com.alibaba.dubbo.rpc.service.GenericException) {
-                com.alibaba.dubbo.rpc.service.GenericException exception = (com.alibaba.dubbo.rpc.service.GenericException) appResponse.getException();
-                try {
-                    String className = exception.getExceptionClass();
-                    Class<?> clazz = ReflectUtils.forName(className);
-                    Throwable targetException = null;
-                    Throwable lastException = null;
-                    try {
-                        targetException = (Throwable) clazz.newInstance();
-                    } catch (Throwable e) {
-                        lastException = e;
-                        for (Constructor<?> constructor : clazz.getConstructors()) {
-                            try {
-                                targetException = (Throwable) constructor.newInstance(new Object[constructor.getParameterTypes().length]);
-                                break;
-                            } catch (Throwable e1) {
-                                lastException = e1;
-                            }
-                        }
-                    }
-                    if (targetException != null) {
-                        try {
-                            Field field = Throwable.class.getDeclaredField("detailMessage");
-                            if (!field.isAccessible()) {
-                                field.setAccessible(true);
-                            }
-                            field.set(targetException, exception.getExceptionMessage());
-                        } catch (Throwable e) {
-                            logger.warn(e.getMessage(), e);
-                        }
-                        appResponse.setException(targetException);
-                    } else if (lastException != null) {
-                        throw lastException;
-                    }
-                } catch (Throwable e) {
-                    throw new RpcException("Can not deserialize exception " + exception.getExceptionClass() + ", message: " + exception.getExceptionMessage(), e);
-                }
-            }
-        }
-    }
-
-    @Override
-    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
-
-    }
-
-    private boolean isCallingGenericImpl(String generic, Invocation invocation) {
-        return ProtocolUtils.isGeneric(generic)
-                && (!$INVOKE.equals(invocation.getMethodName()) && !$INVOKE_ASYNC.equals(invocation.getMethodName()))
-                && invocation instanceof RpcInvocation;
-    }
-
-    private boolean isMakingGenericCall(String generic, Invocation invocation) {
-        return (invocation.getMethodName().equals($INVOKE) || invocation.getMethodName().equals($INVOKE_ASYNC))
-                && invocation.getArguments() != null
-                && invocation.getArguments().length == 3
-                && ProtocolUtils.isGeneric(generic);
-    }
-
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.beanutil.JavaBeanAccessor;
+import org.apache.dubbo.common.beanutil.JavaBeanDescriptor;
+import org.apache.dubbo.common.beanutil.JavaBeanSerializeUtil;
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.common.utils.PojoUtils;
+import org.apache.dubbo.common.utils.ReflectUtils;
+import org.apache.dubbo.rpc.Constants;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.RpcInvocation;
+import org.apache.dubbo.rpc.service.GenericService;
+import org.apache.dubbo.rpc.support.ProtocolUtils;
+import org.apache.dubbo.rpc.support.RpcUtils;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.Field;
+import java.lang.reflect.Method;
+import java.lang.reflect.Type;
+
+import static org.apache.dubbo.common.constants.CommonConstants.$INVOKE;
+import static org.apache.dubbo.common.constants.CommonConstants.$INVOKE_ASYNC;
+import static org.apache.dubbo.common.constants.CommonConstants.GENERIC_PARAMETER_DESC;
+import static org.apache.dubbo.rpc.Constants.GENERIC_KEY;
+
+/**
+ * GenericImplInvokerFilter
+ */
+@Activate(group = CommonConstants.CONSUMER, value = GENERIC_KEY, order = 20000)
+public class GenericImplFilter implements Filter, Filter.Listener {
+
+    private static final Logger logger = LoggerFactory.getLogger(GenericImplFilter.class);
+
+    private static final Class<?>[] GENERIC_PARAMETER_TYPES = new Class<?>[]{String.class, String[].class, Object[].class};
+
+    private static final String GENERIC_IMPL_MARKER = "GENERIC_IMPL";
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        String generic = invoker.getUrl().getParameter(GENERIC_KEY);
+        // calling a generic impl service
+        if (isCallingGenericImpl(generic, invocation)) {
+            RpcInvocation invocation2 = new RpcInvocation(invocation);
+
+            /**
+             * Mark this invocation as a generic impl call, this value will be removed automatically before passing on the wire.
+             * See {@link RpcUtils#sieveUnnecessaryAttachments(Invocation)}
+             */
+            invocation2.put(GENERIC_IMPL_MARKER, true);
+
+            String methodName = invocation2.getMethodName();
+            Class<?>[] parameterTypes = invocation2.getParameterTypes();
+            Object[] arguments = invocation2.getArguments();
+
+            String[] types = new String[parameterTypes.length];
+            for (int i = 0; i < parameterTypes.length; i++) {
+                types[i] = ReflectUtils.getName(parameterTypes[i]);
+            }
+
+            Object[] args;
+            if (ProtocolUtils.isBeanGenericSerialization(generic)) {
+                args = new Object[arguments.length];
+                for (int i = 0; i < arguments.length; i++) {
+                    args[i] = JavaBeanSerializeUtil.serialize(arguments[i], JavaBeanAccessor.METHOD);
+                }
+            } else {
+                args = PojoUtils.generalize(arguments);
+            }
+
+            if (RpcUtils.isReturnTypeFuture(invocation)) {
+                invocation2.setMethodName($INVOKE_ASYNC);
+            } else {
+                invocation2.setMethodName($INVOKE);
+            }
+            invocation2.setParameterTypes(GENERIC_PARAMETER_TYPES);
+            invocation2.setParameterTypesDesc(GENERIC_PARAMETER_DESC);
+            invocation2.setArguments(new Object[]{methodName, types, args});
+            return invoker.invoke(invocation2);
+        }
+        // making a generic call to a normal service
+        else if (isMakingGenericCall(generic, invocation)) {
+
+            Object[] args = (Object[]) invocation.getArguments()[2];
+            if (ProtocolUtils.isJavaGenericSerialization(generic)) {
+
+                for (Object arg : args) {
+                    if (!(byte[].class == arg.getClass())) {
+                        error(generic, byte[].class.getName(), arg.getClass().getName());
+                    }
+                }
+            } else if (ProtocolUtils.isBeanGenericSerialization(generic)) {
+                for (Object arg : args) {
+                    if (!(arg instanceof JavaBeanDescriptor)) {
+                        error(generic, JavaBeanDescriptor.class.getName(), arg.getClass().getName());
+                    }
+                }
+            }
+
+            invocation.setAttachment(
+                    GENERIC_KEY, invoker.getUrl().getParameter(GENERIC_KEY));
+        }
+        return invoker.invoke(invocation);
+    }
+
+    private void error(String generic, String expected, String actual) throws RpcException {
+        throw new RpcException("Generic serialization [" + generic + "] only support message type " + expected + " and your message type is " + actual);
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        String generic = invoker.getUrl().getParameter(GENERIC_KEY);
+        String methodName = invocation.getMethodName();
+        Class<?>[] parameterTypes = invocation.getParameterTypes();
+        Object genericImplMarker = invocation.get(GENERIC_IMPL_MARKER);
+        if (genericImplMarker != null && (boolean) invocation.get(GENERIC_IMPL_MARKER)) {
+            if (!appResponse.hasException()) {
+                Object value = appResponse.getValue();
+                try {
+                    Class<?> invokerInterface = invoker.getInterface();
+                    if (!$INVOKE.equals(methodName) && !$INVOKE_ASYNC.equals(methodName)
+                            && invokerInterface.isAssignableFrom(GenericService.class)) {
+                        try {
+                            // find the real interface from url
+                            String realInterface = invoker.getUrl().getParameter(Constants.INTERFACE);
+                            invokerInterface = ReflectUtils.forName(realInterface);
+                        } catch (Throwable e) {
+                            // ignore
+                        }
+                    }
+
+                    Method method = invokerInterface.getMethod(methodName, parameterTypes);
+                    if (ProtocolUtils.isBeanGenericSerialization(generic)) {
+                        if (value == null) {
+                            appResponse.setValue(value);
+                        } else if (value instanceof JavaBeanDescriptor) {
+                            appResponse.setValue(JavaBeanSerializeUtil.deserialize((JavaBeanDescriptor) value));
+                        } else {
+                            throw new RpcException("The type of result value is " + value.getClass().getName() + " other than " + JavaBeanDescriptor.class.getName() + ", and the result is " + value);
+                        }
+                    } else {
+                        Type[] types = ReflectUtils.getReturnTypes(method);
+                        appResponse.setValue(PojoUtils.realize(value, (Class<?>) types[0], types[1]));
+                    }
+                } catch (NoSuchMethodException e) {
+                    throw new RpcException(e.getMessage(), e);
+                }
+            } else if (appResponse.getException() instanceof com.alibaba.dubbo.rpc.service.GenericException) {
+                com.alibaba.dubbo.rpc.service.GenericException exception = (com.alibaba.dubbo.rpc.service.GenericException) appResponse.getException();
+                try {
+                    String className = exception.getExceptionClass();
+                    Class<?> clazz = ReflectUtils.forName(className);
+                    Throwable targetException = null;
+                    Throwable lastException = null;
+                    try {
+                        targetException = (Throwable) clazz.newInstance();
+                    } catch (Throwable e) {
+                        lastException = e;
+                        for (Constructor<?> constructor : clazz.getConstructors()) {
+                            try {
+                                targetException = (Throwable) constructor.newInstance(new Object[constructor.getParameterTypes().length]);
+                                break;
+                            } catch (Throwable e1) {
+                                lastException = e1;
+                            }
+                        }
+                    }
+                    if (targetException != null) {
+                        try {
+                            Field field = Throwable.class.getDeclaredField("detailMessage");
+                            if (!field.isAccessible()) {
+                                field.setAccessible(true);
+                            }
+                            field.set(targetException, exception.getExceptionMessage());
+                        } catch (Throwable e) {
+                            logger.warn(e.getMessage(), e);
+                        }
+                        appResponse.setException(targetException);
+                    } else if (lastException != null) {
+                        throw lastException;
+                    }
+                } catch (Throwable e) {
+                    throw new RpcException("Can not deserialize exception " + exception.getExceptionClass() + ", message: " + exception.getExceptionMessage(), e);
+                }
+            }
+        }
+    }
+
+    @Override
+    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
+
+    }
+
+    private boolean isCallingGenericImpl(String generic, Invocation invocation) {
+        return ProtocolUtils.isGeneric(generic)
+                && (!$INVOKE.equals(invocation.getMethodName()) && !$INVOKE_ASYNC.equals(invocation.getMethodName()))
+                && invocation instanceof RpcInvocation;
+    }
+
+    private boolean isMakingGenericCall(String generic, Invocation invocation) {
+        return (invocation.getMethodName().equals($INVOKE) || invocation.getMethodName().equals($INVOKE_ASYNC))
+                && invocation.getArguments() != null
+                && invocation.getArguments().length == 3
+                && ProtocolUtils.isGeneric(generic);
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TimeoutFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TimeoutFilter.java
index 1dfd8ed..1d48d7b 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TimeoutFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TimeoutFilter.java
@@ -1,69 +1,69 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.common.logger.LoggerFactory;
-import org.apache.dubbo.rpc.AppResponse;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcContext;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.TimeoutCountDown;
-
-import java.util.Arrays;
-
-import static org.apache.dubbo.common.constants.CommonConstants.TIME_COUNTDOWN_KEY;
-
-/**
- * Log any invocation timeout, but don't stop server from running
- */
-@Activate(group = CommonConstants.PROVIDER)
-public class TimeoutFilter implements Filter, Filter.Listener {
-
-    private static final Logger logger = LoggerFactory.getLogger(TimeoutFilter.class);
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
-        return invoker.invoke(invocation);
-    }
-
-    @Override
-    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
-        Object obj = RpcContext.getContext().get(TIME_COUNTDOWN_KEY);
-        if (obj != null) {
-            TimeoutCountDown countDown = (TimeoutCountDown) obj;
-            if (countDown.isExpired()) {
-                ((AppResponse) appResponse).clear(); // clear response in case of timeout.
-                if (logger.isWarnEnabled()) {
-                    logger.warn("invoke timed out. method: " + invocation.getMethodName() + " arguments: " +
-                            Arrays.toString(invocation.getArguments()) + " , url is " + invoker.getUrl() +
-                            ", invoke elapsed " + countDown.elapsedMillis() + " ms.");
-                }
-            }
-        }
-    }
-
-    @Override
-    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
-
-    }
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.common.logger.LoggerFactory;
+import org.apache.dubbo.rpc.AppResponse;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.TimeoutCountDown;
+
+import java.util.Arrays;
+
+import static org.apache.dubbo.common.constants.CommonConstants.TIME_COUNTDOWN_KEY;
+
+/**
+ * Log any invocation timeout, but don't stop server from running
+ */
+@Activate(group = CommonConstants.PROVIDER)
+public class TimeoutFilter implements Filter, Filter.Listener {
+
+    private static final Logger logger = LoggerFactory.getLogger(TimeoutFilter.class);
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation invocation) throws RpcException {
+        return invoker.invoke(invocation);
+    }
+
+    @Override
+    public void onResponse(Result appResponse, Invoker<?> invoker, Invocation invocation) {
+        Object obj = RpcContext.getContext().get(TIME_COUNTDOWN_KEY);
+        if (obj != null) {
+            TimeoutCountDown countDown = (TimeoutCountDown) obj;
+            if (countDown.isExpired()) {
+                ((AppResponse) appResponse).clear(); // clear response in case of timeout.
+                if (logger.isWarnEnabled()) {
+                    logger.warn("invoke timed out. method: " + invocation.getMethodName() + " arguments: " +
+                            Arrays.toString(invocation.getArguments()) + " , url is " + invoker.getUrl() +
+                            ", invoke elapsed " + countDown.elapsedMillis() + " ms.");
+                }
+            }
+        }
+    }
+
+    @Override
+    public void onError(Throwable t, Invoker<?> invoker, Invocation invocation) {
+
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TokenFilter.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TokenFilter.java
index 811a8e5..5e5d86f 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TokenFilter.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/filter/TokenFilter.java
@@ -1,57 +1,57 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.constants.CommonConstants;
-import org.apache.dubbo.common.extension.Activate;
-import org.apache.dubbo.common.utils.ConfigUtils;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcContext;
-import org.apache.dubbo.rpc.RpcException;
-
-import java.util.Map;
-
-import static org.apache.dubbo.rpc.Constants.TOKEN_KEY;
-
-/**
- * Perform check whether given provider token is matching with remote token or not. If it does not match
- * it will not allow to invoke remote method.
- *
- * @see Filter
- */
-@Activate(group = CommonConstants.PROVIDER, value = TOKEN_KEY)
-public class TokenFilter implements Filter {
-
-    @Override
-    public Result invoke(Invoker<?> invoker, Invocation inv)
-            throws RpcException {
-        String token = invoker.getUrl().getParameter(TOKEN_KEY);
-        if (ConfigUtils.isNotEmpty(token)) {
-            Class<?> serviceType = invoker.getInterface();
-            Map<String, Object> attachments = inv.getObjectAttachments();
-            String remoteToken = (attachments == null ? null : (String) attachments.get(TOKEN_KEY));
-            if (!token.equals(remoteToken)) {
-                throw new RpcException("Invalid token! Forbid invoke remote service " + serviceType + " method " + inv.getMethodName() + "() from consumer " + RpcContext.getContext().getRemoteHost() + " to provider " + RpcContext.getContext().getLocalHost());
-            }
-        }
-        return invoker.invoke(inv);
-    }
-
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.constants.CommonConstants;
+import org.apache.dubbo.common.extension.Activate;
+import org.apache.dubbo.common.utils.ConfigUtils;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.RpcException;
+
+import java.util.Map;
+
+import static org.apache.dubbo.rpc.Constants.TOKEN_KEY;
+
+/**
+ * Perform check whether given provider token is matching with remote token or not. If it does not match
+ * it will not allow to invoke remote method.
+ *
+ * @see Filter
+ */
+@Activate(group = CommonConstants.PROVIDER, value = TOKEN_KEY)
+public class TokenFilter implements Filter {
+
+    @Override
+    public Result invoke(Invoker<?> invoker, Invocation inv)
+            throws RpcException {
+        String token = invoker.getUrl().getParameter(TOKEN_KEY);
+        if (ConfigUtils.isNotEmpty(token)) {
+            Class<?> serviceType = invoker.getInterface();
+            Map<String, Object> attachments = inv.getObjectAttachments();
+            String remoteToken = (attachments == null ? null : (String) attachments.get(TOKEN_KEY));
+            if (!token.equals(remoteToken)) {
+                throw new RpcException("Invalid token! Forbid invoke remote service " + serviceType + " method " + inv.getMethodName() + "() from consumer " + RpcContext.getContext().getRemoteHost() + " to provider " + RpcContext.getContext().getLocalHost());
+            }
+        }
+        return invoker.invoke(inv);
+    }
+
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolListenerWrapper.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolListenerWrapper.java
index 351024a..3461389 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolListenerWrapper.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/protocol/ProtocolListenerWrapper.java
@@ -35,8 +35,8 @@ import java.util.Collections;
 import java.util.List;
 
 import static org.apache.dubbo.common.constants.CommonConstants.EXPORTER_LISTENER_KEY;
-import static org.apache.dubbo.common.constants.CommonConstants.FILTER_BUILDER_KEY;
 import static org.apache.dubbo.common.constants.CommonConstants.INVOKER_LISTENER_KEY;
+import static org.apache.dubbo.common.constants.RegistryConstants.REGISTRY_CLUSTER_TYPE_KEY;
 
 /**
  * ListenerProtocol
@@ -75,7 +75,7 @@ public class ProtocolListenerWrapper implements Protocol {
         }
 
         Invoker<T> invoker = protocol.refer(type, url);
-        if (StringUtils.isEmpty(url.getParameter(FILTER_BUILDER_KEY))) {
+        if (StringUtils.isEmpty(url.getParameter(REGISTRY_CLUSTER_TYPE_KEY))) {
             invoker = new ListenerInvokerWrapper<>(invoker,
                     Collections.unmodifiableList(
                             ExtensionLoader.getExtensionLoader(InvokerListener.class)
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/support/DefaultFilterChainBuilder.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/support/DefaultFilterChainBuilder.java
deleted file mode 100644
index 24b350b..0000000
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/support/DefaultFilterChainBuilder.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * 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.rpc.support;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.extension.ExtensionLoader;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.FilterChainBuilder;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.ListenableFilter;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-
-import java.util.List;
-
-public class DefaultFilterChainBuilder implements FilterChainBuilder {
-
-    @Override
-    public <T> Invoker<T> buildInvokerChain(final Invoker<T> invoker, String key, String group) {
-        Invoker<T> last = invoker;
-        List<Filter> filters = ExtensionLoader.getExtensionLoader(Filter.class).getActivateExtension(invoker.getUrl(), key, group);
-
-        if (!filters.isEmpty()) {
-            for (int i = filters.size() - 1; i >= 0; i--) {
-                final Filter filter = filters.get(i);
-                final Invoker<T> next = last;
-                last = new Invoker<T>() {
-
-                    @Override
-                    public Class<T> getInterface() {
-                        return invoker.getInterface();
-                    }
-
-                    @Override
-                    public URL getUrl() {
-                        return invoker.getUrl();
-                    }
-
-                    @Override
-                    public boolean isAvailable() {
-                        return invoker.isAvailable();
-                    }
-
-                    @Override
-                    public Result invoke(Invocation invocation) throws RpcException {
-                        Result asyncResult;
-                        try {
-                            asyncResult = filter.invoke(next, invocation);
-                        } catch (Exception e) {
-                            if (filter instanceof ListenableFilter) {
-                                ListenableFilter listenableFilter = ((ListenableFilter) filter);
-                                try {
-                                    Filter.Listener listener = listenableFilter.listener(invocation);
-                                    if (listener != null) {
-                                        listener.onError(e, invoker, invocation);
-                                    }
-                                } finally {
-                                    listenableFilter.removeListener(invocation);
-                                }
-                            } else if (filter instanceof Filter.Listener) {
-                                Filter.Listener listener = (Filter.Listener) filter;
-                                listener.onError(e, invoker, invocation);
-                            }
-                            throw e;
-                        } finally {
-
-                        }
-                        return asyncResult.whenCompleteWithContext((r, t) -> {
-                            if (filter instanceof ListenableFilter) {
-                                ListenableFilter listenableFilter = ((ListenableFilter) filter);
-                                Filter.Listener listener = listenableFilter.listener(invocation);
-                                try {
-                                    if (listener != null) {
-                                        if (t == null) {
-                                            listener.onResponse(r, invoker, invocation);
-                                        } else {
-                                            listener.onError(t, invoker, invocation);
-                                        }
-                                    }
-                                } finally {
-                                    listenableFilter.removeListener(invocation);
-                                }
-                            } else if (filter instanceof Filter.Listener) {
-                                Filter.Listener listener = (Filter.Listener) filter;
-                                if (t == null) {
-                                    listener.onResponse(r, invoker, invocation);
-                                } else {
-                                    listener.onError(t, invoker, invocation);
-                                }
-                            }
-                        });
-                    }
-
-                    @Override
-                    public void destroy() {
-                        invoker.destroy();
-                    }
-
-                    @Override
-                    public String toString() {
-                        return invoker.toString();
-                    }
-                };
-            }
-        }
-
-        return last;
-    }
-
-}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Filter b/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Filter
index 3d6da08..5255f55 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Filter
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Filter
@@ -1,15 +1,15 @@
-echo=org.apache.dubbo.rpc.filter.EchoFilter
-generic=org.apache.dubbo.rpc.filter.GenericFilter
-genericimpl=org.apache.dubbo.rpc.filter.GenericImplFilter
-token=org.apache.dubbo.rpc.filter.TokenFilter
-accesslog=org.apache.dubbo.rpc.filter.AccessLogFilter
-activelimit=org.apache.dubbo.rpc.filter.ActiveLimitFilter
-classloader=org.apache.dubbo.rpc.filter.ClassLoaderFilter
-context=org.apache.dubbo.rpc.filter.ContextFilter
-consumercontext=org.apache.dubbo.rpc.filter.ConsumerContextFilter
-exception=org.apache.dubbo.rpc.filter.ExceptionFilter
-executelimit=org.apache.dubbo.rpc.filter.ExecuteLimitFilter
-deprecated=org.apache.dubbo.rpc.filter.DeprecatedFilter
-compatible=org.apache.dubbo.rpc.filter.CompatibleFilter
-timeout=org.apache.dubbo.rpc.filter.TimeoutFilter
+echo=org.apache.dubbo.rpc.filter.EchoFilter
+generic=org.apache.dubbo.rpc.filter.GenericFilter
+genericimpl=org.apache.dubbo.rpc.filter.GenericImplFilter
+token=org.apache.dubbo.rpc.filter.TokenFilter
+accesslog=org.apache.dubbo.rpc.filter.AccessLogFilter
+activelimit=org.apache.dubbo.rpc.filter.ActiveLimitFilter
+classloader=org.apache.dubbo.rpc.filter.ClassLoaderFilter
+context=org.apache.dubbo.rpc.filter.ContextFilter
+consumercontext=org.apache.dubbo.rpc.filter.ConsumerContextFilter
+exception=org.apache.dubbo.rpc.filter.ExceptionFilter
+executelimit=org.apache.dubbo.rpc.filter.ExecuteLimitFilter
+deprecated=org.apache.dubbo.rpc.filter.DeprecatedFilter
+compatible=org.apache.dubbo.rpc.filter.CompatibleFilter
+timeout=org.apache.dubbo.rpc.filter.TimeoutFilter
 tps=org.apache.dubbo.rpc.filter.TpsLimitFilter
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.FilterChainBuilder b/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.FilterChainBuilder
deleted file mode 100644
index b3a9854..0000000
--- a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.FilterChainBuilder
+++ /dev/null
@@ -1 +0,0 @@
-default=org.apache.dubbo.rpc.support.DefaultFilterChainBuilder
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol b/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol
index 689e812..3830dd1 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/resources/META-INF/dubbo/internal/org.apache.dubbo.rpc.Protocol
@@ -1,3 +1,2 @@
-filter=org.apache.dubbo.rpc.protocol.ProtocolFilterWrapper
 listener=org.apache.dubbo.rpc.protocol.ProtocolListenerWrapper
 mock=org.apache.dubbo.rpc.support.MockProtocol
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/AccessLogFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/AccessLogFilterTest.java
index 5bfcb4b..353f9a3 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/AccessLogFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/AccessLogFilterTest.java
@@ -1,85 +1,85 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.utils.LogUtil;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.support.AccessLogData;
-import org.apache.dubbo.rpc.support.MockInvocation;
-import org.apache.dubbo.rpc.support.MyInvoker;
-
-import org.junit.jupiter.api.Test;
-
-import java.lang.reflect.Field;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertFalse;
-import static org.junit.jupiter.api.Assertions.assertTrue;
-
-/**
- * AccessLogFilterTest.java
- */
-public class AccessLogFilterTest {
-
-    Filter accessLogFilter = new AccessLogFilter();
-
-    // Test filter won't throw an exception
-    @Test
-    public void testInvokeException() {
-        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(null);
-        Invocation invocation = new MockInvocation();
-        LogUtil.start();
-        accessLogFilter.invoke(invoker, invocation);
-        assertEquals(1, LogUtil.findMessage("Exception in AccessLogFilter of service"));
-        LogUtil.stop();
-    }
-
-    // TODO how to assert thread action
-    @Test
-    @SuppressWarnings("unchecked")
-    public void testDefault() throws NoSuchFieldException, IllegalAccessException {
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1");
-        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(url);
-        Invocation invocation = new MockInvocation();
-
-        Field field = AccessLogFilter.class.getDeclaredField("LOG_ENTRIES");
-        field.setAccessible(true);
-        assertTrue(((Map) field.get(AccessLogFilter.class)).isEmpty());
-
-        accessLogFilter.invoke(invoker, invocation);
-
-        Map<String, Set<AccessLogData>> logs = (Map<String, Set<AccessLogData>>) field.get(AccessLogFilter.class);
-        assertFalse(logs.isEmpty());
-        assertFalse(logs.get("true").isEmpty());
-        AccessLogData log = logs.get("true").iterator().next();
-        assertEquals("org.apache.dubbo.rpc.support.DemoService", log.getServiceName());
-    }
-
-    @Test
-    public void testCustom() {
-        URL url = URL.valueOf("test://test:11/test?accesslog=custom-access.log");
-        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(url);
-        Invocation invocation = new MockInvocation();
-        accessLogFilter.invoke(invoker, invocation);
-    }
-
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.utils.LogUtil;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.support.AccessLogData;
+import org.apache.dubbo.rpc.support.MockInvocation;
+import org.apache.dubbo.rpc.support.MyInvoker;
+
+import org.junit.jupiter.api.Test;
+
+import java.lang.reflect.Field;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+/**
+ * AccessLogFilterTest.java
+ */
+public class AccessLogFilterTest {
+
+    Filter accessLogFilter = new AccessLogFilter();
+
+    // Test filter won't throw an exception
+    @Test
+    public void testInvokeException() {
+        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(null);
+        Invocation invocation = new MockInvocation();
+        LogUtil.start();
+        accessLogFilter.invoke(invoker, invocation);
+        assertEquals(1, LogUtil.findMessage("Exception in AccessLogFilter of service"));
+        LogUtil.stop();
+    }
+
+    // TODO how to assert thread action
+    @Test
+    @SuppressWarnings("unchecked")
+    public void testDefault() throws NoSuchFieldException, IllegalAccessException {
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1");
+        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(url);
+        Invocation invocation = new MockInvocation();
+
+        Field field = AccessLogFilter.class.getDeclaredField("LOG_ENTRIES");
+        field.setAccessible(true);
+        assertTrue(((Map) field.get(AccessLogFilter.class)).isEmpty());
+
+        accessLogFilter.invoke(invoker, invocation);
+
+        Map<String, Set<AccessLogData>> logs = (Map<String, Set<AccessLogData>>) field.get(AccessLogFilter.class);
+        assertFalse(logs.isEmpty());
+        assertFalse(logs.get("true").isEmpty());
+        AccessLogData log = logs.get("true").iterator().next();
+        assertEquals("org.apache.dubbo.rpc.support.DemoService", log.getServiceName());
+    }
+
+    @Test
+    public void testCustom() {
+        URL url = URL.valueOf("test://test:11/test?accesslog=custom-access.log");
+        Invoker<AccessLogFilterTest> invoker = new MyInvoker<AccessLogFilterTest>(url);
+        Invocation invocation = new MockInvocation();
+        accessLogFilter.invoke(invoker, invocation);
+    }
+
 }
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ActiveLimitFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ActiveLimitFilterTest.java
index 2268bf2..7f741f1 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ActiveLimitFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ActiveLimitFilterTest.java
@@ -1,226 +1,226 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.RpcStatus;
-import org.apache.dubbo.rpc.support.BlockMyInvoker;
-import org.apache.dubbo.rpc.support.MockInvocation;
-import org.apache.dubbo.rpc.support.MyInvoker;
-import org.apache.dubbo.rpc.support.RuntimeExceptionInvoker;
-
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.junit.jupiter.api.Assertions.assertNotSame;
-import static org.junit.jupiter.api.Assertions.fail;
-
-/**
- * ActiveLimitFilterTest.java
- */
-public class ActiveLimitFilterTest {
-
-    ActiveLimitFilter activeLimitFilter = new ActiveLimitFilter();
-
-    @Test
-    public void testInvokeNoActives() {
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
-        Invoker<ActiveLimitFilterTest> invoker = new MyInvoker<ActiveLimitFilterTest>(url);
-        Invocation invocation = new MockInvocation();
-        activeLimitFilter.invoke(invoker, invocation);
-    }
-
-    @Test
-    public void testInvokeLessActives() {
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=10");
-        Invoker<ActiveLimitFilterTest> invoker = new MyInvoker<ActiveLimitFilterTest>(url);
-        Invocation invocation = new MockInvocation();
-        activeLimitFilter.invoke(invoker, invocation);
-    }
-
-    @Test
-    public void testInvokeGreaterActives() {
-        AtomicInteger count = new AtomicInteger(0);
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=1&timeout=1");
-        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, 100);
-        final Invocation invocation = new MockInvocation();
-        final CountDownLatch latch = new CountDownLatch(1);
-        for (int i = 0; i < 100; i++) {
-            Thread thread = new Thread(new Runnable() {
-
-                public void run() {
-                    try {
-                        latch.await();
-                    } catch (InterruptedException e) {
-                        e.printStackTrace();
-                    }
-                    for (int i = 0; i < 100; i++) {
-                        try {
-                            activeLimitFilter.invoke(invoker, invocation);
-                        } catch (RpcException expected) {
-                            count.incrementAndGet();
-                        }
-                    }
-                }
-            });
-            thread.start();
-        }
-        latch.countDown();
-
-        try {
-            Thread.sleep(1000);
-        } catch (InterruptedException e) {
-            e.printStackTrace();
-        }
-        assertNotSame(0, count.intValue());
-    }
-
-    @Test
-    public void testInvokeTimeOut() throws Exception {
-        int totalThread = 100;
-        int maxActives = 10;
-        long timeout = 1;
-        long blockTime = 100;
-        AtomicInteger count = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(1);
-        final CountDownLatch latchBlocking = new CountDownLatch(totalThread);
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=" + maxActives + "&timeout=" + timeout);
-        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, blockTime);
-        final Invocation invocation = new MockInvocation();
-        RpcStatus.removeStatus(url);
-        RpcStatus.removeStatus(url, invocation.getMethodName());
-        for (int i = 0; i < totalThread; i++) {
-            Thread thread = new Thread(new Runnable() {
-                public void run() {
-                    try {
-                        try {
-                            latch.await();
-                        } catch (InterruptedException e) {
-                            e.printStackTrace();
-                        }
-                        try {
-                            Result asyncResult = activeLimitFilter.invoke(invoker, invocation);
-                            Result result = asyncResult.get();
-                            activeLimitFilter.onResponse(result, invoker, invocation);
-                        } catch (RpcException expected) {
-                            count.incrementAndGet();
-                        } catch (Exception e) {
-                            fail();
-                        }
-                    } finally {
-                        latchBlocking.countDown();
-                    }
-                }
-            });
-            thread.start();
-        }
-        latch.countDown();
-
-        try {
-            latchBlocking.await();
-        } catch (InterruptedException e) {
-            e.printStackTrace();
-        }
-        assertEquals(90, count.intValue());
-    }
-
-    @Test
-    public void testInvokeNotTimeOut() throws Exception {
-        int totalThread = 100;
-        int maxActives = 10;
-        long timeout = 1000;
-        long blockTime = 0;
-        AtomicInteger count = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(1);
-        final CountDownLatch latchBlocking = new CountDownLatch(totalThread);
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=" + maxActives + "&timeout=" + timeout);
-        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, blockTime);
-        final Invocation invocation = new MockInvocation();
-        for (int i = 0; i < totalThread; i++) {
-            Thread thread = new Thread(new Runnable() {
-                public void run() {
-                    try {
-                        try {
-                            latch.await();
-                        } catch (InterruptedException e) {
-                            e.printStackTrace();
-                        }
-                        try {
-                            Result asyncResult = activeLimitFilter.invoke(invoker, invocation);
-                            Result result = asyncResult.get();
-                            activeLimitFilter.onResponse(result, invoker, invocation);
-                        } catch (RpcException expected) {
-                            count.incrementAndGet();
-                            activeLimitFilter.onError(expected, invoker, invocation);
-                        } catch (Exception e) {
-                            fail();
-                        }
-                    } finally {
-                        latchBlocking.countDown();
-                    }
-                }
-            });
-            thread.start();
-        }
-        latch.countDown();
-
-        try {
-            latchBlocking.await();
-        } catch (InterruptedException e) {
-            e.printStackTrace();
-        }
-        assertEquals(0, count.intValue());
-    }
-
-    @Test
-    public void testInvokeRuntimeException() {
-        Assertions.assertThrows(RuntimeException.class, () -> {
-            URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
-            Invoker<ActiveLimitFilterTest> invoker = new RuntimeExceptionInvoker(url);
-            Invocation invocation = new MockInvocation();
-            RpcStatus count = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
-            int beforeExceptionActiveCount = count.getActive();
-            activeLimitFilter.invoke(invoker, invocation);
-            int afterExceptionActiveCount = count.getActive();
-            assertEquals(beforeExceptionActiveCount, afterExceptionActiveCount, "After exception active count should be same");
-        });
-    }
-
-    @Test
-    public void testInvokeRuntimeExceptionWithActiveCountMatch() {
-        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
-        Invoker<ActiveLimitFilterTest> invoker = new RuntimeExceptionInvoker(url);
-        Invocation invocation = new MockInvocation();
-        RpcStatus count = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
-        int beforeExceptionActiveCount = count.getActive();
-        try {
-            activeLimitFilter.invoke(invoker, invocation);
-        } catch (RuntimeException ex) {
-            activeLimitFilter.onError(ex, invoker, invocation);
-            int afterExceptionActiveCount = count.getActive();
-            assertEquals(beforeExceptionActiveCount, afterExceptionActiveCount, "After exception active count should be same");
-        }
-    }
-}
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.RpcStatus;
+import org.apache.dubbo.rpc.support.BlockMyInvoker;
+import org.apache.dubbo.rpc.support.MockInvocation;
+import org.apache.dubbo.rpc.support.MyInvoker;
+import org.apache.dubbo.rpc.support.RuntimeExceptionInvoker;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.fail;
+
+/**
+ * ActiveLimitFilterTest.java
+ */
+public class ActiveLimitFilterTest {
+
+    ActiveLimitFilter activeLimitFilter = new ActiveLimitFilter();
+
+    @Test
+    public void testInvokeNoActives() {
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
+        Invoker<ActiveLimitFilterTest> invoker = new MyInvoker<ActiveLimitFilterTest>(url);
+        Invocation invocation = new MockInvocation();
+        activeLimitFilter.invoke(invoker, invocation);
+    }
+
+    @Test
+    public void testInvokeLessActives() {
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=10");
+        Invoker<ActiveLimitFilterTest> invoker = new MyInvoker<ActiveLimitFilterTest>(url);
+        Invocation invocation = new MockInvocation();
+        activeLimitFilter.invoke(invoker, invocation);
+    }
+
+    @Test
+    public void testInvokeGreaterActives() {
+        AtomicInteger count = new AtomicInteger(0);
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=1&timeout=1");
+        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, 100);
+        final Invocation invocation = new MockInvocation();
+        final CountDownLatch latch = new CountDownLatch(1);
+        for (int i = 0; i < 100; i++) {
+            Thread thread = new Thread(new Runnable() {
+
+                public void run() {
+                    try {
+                        latch.await();
+                    } catch (InterruptedException e) {
+                        e.printStackTrace();
+                    }
+                    for (int i = 0; i < 100; i++) {
+                        try {
+                            activeLimitFilter.invoke(invoker, invocation);
+                        } catch (RpcException expected) {
+                            count.incrementAndGet();
+                        }
+                    }
+                }
+            });
+            thread.start();
+        }
+        latch.countDown();
+
+        try {
+            Thread.sleep(1000);
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+        assertNotSame(0, count.intValue());
+    }
+
+    @Test
+    public void testInvokeTimeOut() throws Exception {
+        int totalThread = 100;
+        int maxActives = 10;
+        long timeout = 1;
+        long blockTime = 100;
+        AtomicInteger count = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch latchBlocking = new CountDownLatch(totalThread);
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=" + maxActives + "&timeout=" + timeout);
+        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, blockTime);
+        final Invocation invocation = new MockInvocation();
+        RpcStatus.removeStatus(url);
+        RpcStatus.removeStatus(url, invocation.getMethodName());
+        for (int i = 0; i < totalThread; i++) {
+            Thread thread = new Thread(new Runnable() {
+                public void run() {
+                    try {
+                        try {
+                            latch.await();
+                        } catch (InterruptedException e) {
+                            e.printStackTrace();
+                        }
+                        try {
+                            Result asyncResult = activeLimitFilter.invoke(invoker, invocation);
+                            Result result = asyncResult.get();
+                            activeLimitFilter.onResponse(result, invoker, invocation);
+                        } catch (RpcException expected) {
+                            count.incrementAndGet();
+                        } catch (Exception e) {
+                            fail();
+                        }
+                    } finally {
+                        latchBlocking.countDown();
+                    }
+                }
+            });
+            thread.start();
+        }
+        latch.countDown();
+
+        try {
+            latchBlocking.await();
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+        assertEquals(90, count.intValue());
+    }
+
+    @Test
+    public void testInvokeNotTimeOut() throws Exception {
+        int totalThread = 100;
+        int maxActives = 10;
+        long timeout = 1000;
+        long blockTime = 0;
+        AtomicInteger count = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch latchBlocking = new CountDownLatch(totalThread);
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=" + maxActives + "&timeout=" + timeout);
+        final Invoker<ActiveLimitFilterTest> invoker = new BlockMyInvoker<ActiveLimitFilterTest>(url, blockTime);
+        final Invocation invocation = new MockInvocation();
+        for (int i = 0; i < totalThread; i++) {
+            Thread thread = new Thread(new Runnable() {
+                public void run() {
+                    try {
+                        try {
+                            latch.await();
+                        } catch (InterruptedException e) {
+                            e.printStackTrace();
+                        }
+                        try {
+                            Result asyncResult = activeLimitFilter.invoke(invoker, invocation);
+                            Result result = asyncResult.get();
+                            activeLimitFilter.onResponse(result, invoker, invocation);
+                        } catch (RpcException expected) {
+                            count.incrementAndGet();
+                            activeLimitFilter.onError(expected, invoker, invocation);
+                        } catch (Exception e) {
+                            fail();
+                        }
+                    } finally {
+                        latchBlocking.countDown();
+                    }
+                }
+            });
+            thread.start();
+        }
+        latch.countDown();
+
+        try {
+            latchBlocking.await();
+        } catch (InterruptedException e) {
+            e.printStackTrace();
+        }
+        assertEquals(0, count.intValue());
+    }
+
+    @Test
+    public void testInvokeRuntimeException() {
+        Assertions.assertThrows(RuntimeException.class, () -> {
+            URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
+            Invoker<ActiveLimitFilterTest> invoker = new RuntimeExceptionInvoker(url);
+            Invocation invocation = new MockInvocation();
+            RpcStatus count = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
+            int beforeExceptionActiveCount = count.getActive();
+            activeLimitFilter.invoke(invoker, invocation);
+            int afterExceptionActiveCount = count.getActive();
+            assertEquals(beforeExceptionActiveCount, afterExceptionActiveCount, "After exception active count should be same");
+        });
+    }
+
+    @Test
+    public void testInvokeRuntimeExceptionWithActiveCountMatch() {
+        URL url = URL.valueOf("test://test:11/test?accesslog=true&group=dubbo&version=1.1&actives=0");
+        Invoker<ActiveLimitFilterTest> invoker = new RuntimeExceptionInvoker(url);
+        Invocation invocation = new MockInvocation();
+        RpcStatus count = RpcStatus.getStatus(invoker.getUrl(), invocation.getMethodName());
+        int beforeExceptionActiveCount = count.getActive();
+        try {
+            activeLimitFilter.invoke(invoker, invocation);
+        } catch (RuntimeException ex) {
+            activeLimitFilter.onError(ex, invoker, invocation);
+            int afterExceptionActiveCount = count.getActive();
+            assertEquals(beforeExceptionActiveCount, afterExceptionActiveCount, "After exception active count should be same");
+        }
+    }
+}
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/CompatibleFilterFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/CompatibleFilterFilterTest.java
index bc05c98..c8f6a7f 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/CompatibleFilterFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/CompatibleFilterFilterTest.java
@@ -1,174 +1,174 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.rpc.AppResponse;
-import org.apache.dubbo.rpc.AsyncRpcResult;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.support.DemoService;
-import org.apache.dubbo.rpc.support.Type;
-
-import org.junit.jupiter.api.AfterEach;
-import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
-
-import static org.junit.jupiter.api.Assertions.assertArrayEquals;
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.mock;
-
-/**
- * CompatibleFilterTest.java
- */
-public class CompatibleFilterFilterTest {
-    private CompatibleFilter compatibleFilter = new CompatibleFilter();
-    private Invocation invocation;
-    private Invoker invoker;
-
-    @AfterEach
-    public void tearDown() {
-        Mockito.reset(invocation, invoker);
-    }
-
-    @Test
-    public void testInvokerGeneric() {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("$enumlength");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = compatibleFilter.invoke(invoker, invocation);
-        assertEquals(filterResult, result);
-    }
-
-    @Test
-    public void testResulthasException() {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("enumlength");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setException(new RuntimeException());
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = compatibleFilter.invoke(invoker, invocation);
-        assertEquals(filterResult, result);
-    }
-
-    @Test
-    public void testInvokerJsonPojoSerialization() throws Exception {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("enumlength");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Type[].class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(AsyncRpcResult.newDefaultAsyncResult(result, invocation));
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1&serialization=json");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result asyncResult = compatibleFilter.invoke(invoker, invocation);
-        AppResponse appResponse = (AppResponse) asyncResult.get();
-        compatibleFilter.onResponse(appResponse, invoker, invocation);
-        assertEquals(Type.High, appResponse.getValue());
-    }
-
-    @Test
-    public void testInvokerNonJsonEnumSerialization() throws Exception {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("enumlength");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Type[].class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(AsyncRpcResult.newDefaultAsyncResult(result, invocation));
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result asyncResult = compatibleFilter.invoke(invoker, invocation);
-        AppResponse appResponse = (AppResponse) asyncResult.get();
-        compatibleFilter.onResponse(appResponse, invoker, invocation);
-        assertEquals(Type.High, appResponse.getValue());
-    }
-
-    @Test
-    public void testInvokerNonJsonNonPojoSerialization() {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("echo");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{String.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue(new String[]{"High"});
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = compatibleFilter.invoke(invoker, invocation);
-        assertArrayEquals(new String[]{"High"}, (String[]) filterResult.getValue());
-    }
-
-    @Test
-    public void testInvokerNonJsonPojoSerialization() {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("echo");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{String.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("hello");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = compatibleFilter.invoke(invoker, invocation);
-        assertEquals("hello", filterResult.getValue());
-    }
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.rpc.AppResponse;
+import org.apache.dubbo.rpc.AsyncRpcResult;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.support.DemoService;
+import org.apache.dubbo.rpc.support.Type;
+
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertArrayEquals;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.mock;
+
+/**
+ * CompatibleFilterTest.java
+ */
+public class CompatibleFilterFilterTest {
+    private CompatibleFilter compatibleFilter = new CompatibleFilter();
+    private Invocation invocation;
+    private Invoker invoker;
+
+    @AfterEach
+    public void tearDown() {
+        Mockito.reset(invocation, invoker);
+    }
+
+    @Test
+    public void testInvokerGeneric() {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("$enumlength");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = compatibleFilter.invoke(invoker, invocation);
+        assertEquals(filterResult, result);
+    }
+
+    @Test
+    public void testResulthasException() {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("enumlength");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setException(new RuntimeException());
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = compatibleFilter.invoke(invoker, invocation);
+        assertEquals(filterResult, result);
+    }
+
+    @Test
+    public void testInvokerJsonPojoSerialization() throws Exception {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("enumlength");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Type[].class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(AsyncRpcResult.newDefaultAsyncResult(result, invocation));
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1&serialization=json");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result asyncResult = compatibleFilter.invoke(invoker, invocation);
+        AppResponse appResponse = (AppResponse) asyncResult.get();
+        compatibleFilter.onResponse(appResponse, invoker, invocation);
+        assertEquals(Type.High, appResponse.getValue());
+    }
+
+    @Test
+    public void testInvokerNonJsonEnumSerialization() throws Exception {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("enumlength");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Type[].class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(AsyncRpcResult.newDefaultAsyncResult(result, invocation));
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result asyncResult = compatibleFilter.invoke(invoker, invocation);
+        AppResponse appResponse = (AppResponse) asyncResult.get();
+        compatibleFilter.onResponse(appResponse, invoker, invocation);
+        assertEquals(Type.High, appResponse.getValue());
+    }
+
+    @Test
+    public void testInvokerNonJsonNonPojoSerialization() {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("echo");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{String.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue(new String[]{"High"});
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = compatibleFilter.invoke(invoker, invocation);
+        assertArrayEquals(new String[]{"High"}, (String[]) filterResult.getValue());
+    }
+
+    @Test
+    public void testInvokerNonJsonPojoSerialization() {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("echo");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{String.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("hello");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = compatibleFilter.invoke(invoker, invocation);
+        assertEquals("hello", filterResult.getValue());
+    }
 }
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ContextFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ContextFilterTest.java
index c38cf1b..ed65129 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ContextFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ContextFilterTest.java
@@ -1,76 +1,76 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.rpc.AppResponse;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcContext;
-import org.apache.dubbo.rpc.support.DemoService;
-import org.apache.dubbo.rpc.support.MockInvocation;
-import org.apache.dubbo.rpc.support.MyInvoker;
-
-import org.junit.jupiter.api.Test;
-
-import static org.junit.jupiter.api.Assertions.assertNull;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.mock;
-
-/**
- * ContextFilterTest.java
- * TODO need to enhance assertion
- */
-public class ContextFilterTest {
-
-    Filter contextFilter = new ContextFilter();
-    Invoker<DemoService> invoker;
-    Invocation invocation;
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testSetContext() {
-        invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("$enumlength");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-        given(invocation.getObjectAttachments()).willReturn(null);
-
-        invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        contextFilter.invoke(invoker, invocation);
-        assertNull(RpcContext.getContext().getInvoker());
-    }
-
-    @Test
-    public void testWithAttachments() {
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        Invoker<DemoService> invoker = new MyInvoker<DemoService>(url);
-        Invocation invocation = new MockInvocation();
-        Result result = contextFilter.invoke(invoker, invocation);
-        assertNull(RpcContext.getContext().getInvoker());
-    }
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.rpc.AppResponse;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.support.DemoService;
+import org.apache.dubbo.rpc.support.MockInvocation;
+import org.apache.dubbo.rpc.support.MyInvoker;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.mock;
+
+/**
+ * ContextFilterTest.java
+ * TODO need to enhance assertion
+ */
+public class ContextFilterTest {
+
+    Filter contextFilter = new ContextFilter();
+    Invoker<DemoService> invoker;
+    Invocation invocation;
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testSetContext() {
+        invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("$enumlength");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+        given(invocation.getObjectAttachments()).willReturn(null);
+
+        invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        contextFilter.invoke(invoker, invocation);
+        assertNull(RpcContext.getContext().getInvoker());
+    }
+
+    @Test
+    public void testWithAttachments() {
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        Invoker<DemoService> invoker = new MyInvoker<DemoService>(url);
+        Invocation invocation = new MockInvocation();
+        Result result = contextFilter.invoke(invoker, invocation);
+        assertNull(RpcContext.getContext().getInvoker());
+    }
 }
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/DeprecatedFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/DeprecatedFilterTest.java
index 1c7b4d4..32835ff 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/DeprecatedFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/DeprecatedFilterTest.java
@@ -1,49 +1,48 @@
-/*
- * 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.rpc.filter;
-
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.common.utils.LogUtil;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.support.DemoService;
-import org.apache.dubbo.rpc.support.MockInvocation;
-import org.apache.dubbo.rpc.support.MyInvoker;
-
-import org.junit.jupiter.api.Test;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-
-import static org.apache.dubbo.rpc.Constants.DEPRECATED_KEY;
-
-/**
- * DeprecatedFilterTest.java
- */
-public class DeprecatedFilterTest {
-
-    Filter deprecatedFilter = new DeprecatedFilter();
-
-    @Test
-    public void testDeprecatedFilter() {
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1&echo." + DEPRECATED_KEY + "=true");
-        LogUtil.start();
-        deprecatedFilter.invoke(new MyInvoker<DemoService>(url), new MockInvocation());
-        assertEquals(1,
-                LogUtil.findMessage("The service method org.apache.dubbo.rpc.support.DemoService.echo(String) is DEPRECATED"));
-        LogUtil.stop();
-    }
+/*
+ * 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.rpc.filter;
+
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.common.utils.LogUtil;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.support.DemoService;
+import org.apache.dubbo.rpc.support.MockInvocation;
+import org.apache.dubbo.rpc.support.MyInvoker;
+
+import org.junit.jupiter.api.Test;
+
+import static org.apache.dubbo.rpc.Constants.DEPRECATED_KEY;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+
+/**
+ * DeprecatedFilterTest.java
+ */
+public class DeprecatedFilterTest {
+
+    Filter deprecatedFilter = new DeprecatedFilter();
+
+    @Test
+    public void testDeprecatedFilter() {
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1&echo." + DEPRECATED_KEY + "=true");
+        LogUtil.start();
+        deprecatedFilter.invoke(new MyInvoker<DemoService>(url), new MockInvocation());
+        assertEquals(1,
+                LogUtil.findMessage("The service method org.apache.dubbo.rpc.support.DemoService.echo(String) is DEPRECATED"));
+        LogUtil.stop();
+    }
 }
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/EchoFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/EchoFilterTest.java
index 996fdf6..3ed2df2 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/EchoFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/EchoFilterTest.java
@@ -1,80 +1,80 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.URL;
-import org.apache.dubbo.rpc.AppResponse;
-import org.apache.dubbo.rpc.Filter;
-import org.apache.dubbo.rpc.Invocation;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.support.DemoService;
-
-import org.junit.jupiter.api.Test;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.mock;
-
-public class EchoFilterTest {
-
-    Filter echoFilter = new EchoFilter();
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testEcho() {
-        Invocation invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("$echo");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-        given(invocation.getObjectAttachments()).willReturn(null);
-
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = echoFilter.invoke(invoker, invocation);
-        assertEquals("hello", filterResult.getValue());
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testNonEcho() {
-        Invocation invocation = mock(Invocation.class);
-        given(invocation.getMethodName()).willReturn("echo");
-        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
-        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
-        given(invocation.getObjectAttachments()).willReturn(null);
-
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        given(invoker.isAvailable()).willReturn(true);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        AppResponse result = new AppResponse();
-        result.setValue("High");
-        given(invoker.invoke(invocation)).willReturn(result);
-        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
-        given(invoker.getUrl()).willReturn(url);
-
-        Result filterResult = echoFilter.invoke(invoker, invocation);
-        assertEquals("High", filterResult.getValue());
-    }
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.URL;
+import org.apache.dubbo.rpc.AppResponse;
+import org.apache.dubbo.rpc.Filter;
+import org.apache.dubbo.rpc.Invocation;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.support.DemoService;
+
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.mock;
+
+public class EchoFilterTest {
+
+    Filter echoFilter = new EchoFilter();
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testEcho() {
+        Invocation invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("$echo");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+        given(invocation.getObjectAttachments()).willReturn(null);
+
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = echoFilter.invoke(invoker, invocation);
+        assertEquals("hello", filterResult.getValue());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testNonEcho() {
+        Invocation invocation = mock(Invocation.class);
+        given(invocation.getMethodName()).willReturn("echo");
+        given(invocation.getParameterTypes()).willReturn(new Class<?>[]{Enum.class});
+        given(invocation.getArguments()).willReturn(new Object[]{"hello"});
+        given(invocation.getObjectAttachments()).willReturn(null);
+
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        given(invoker.isAvailable()).willReturn(true);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        AppResponse result = new AppResponse();
+        result.setValue("High");
+        given(invoker.invoke(invocation)).willReturn(result);
+        URL url = URL.valueOf("test://test:11/test?group=dubbo&version=1.1");
+        given(invoker.getUrl()).willReturn(url);
+
+        Result filterResult = echoFilter.invoke(invoker, invocation);
+        assertEquals("High", filterResult.getValue());
+    }
 }
\ No newline at end of file
diff --git a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ExceptionFilterTest.java b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ExceptionFilterTest.java
index 2f32ff6..b8ebd84 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ExceptionFilterTest.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/test/java/org/apache/dubbo/rpc/filter/ExceptionFilterTest.java
@@ -1,139 +1,139 @@
-/*
- * 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.rpc.filter;
-
-import org.apache.dubbo.common.logger.Logger;
-import org.apache.dubbo.rpc.AppResponse;
-import org.apache.dubbo.rpc.AsyncRpcResult;
-import org.apache.dubbo.rpc.Invoker;
-import org.apache.dubbo.rpc.Result;
-import org.apache.dubbo.rpc.RpcContext;
-import org.apache.dubbo.rpc.RpcException;
-import org.apache.dubbo.rpc.RpcInvocation;
-import org.apache.dubbo.rpc.support.DemoService;
-import org.apache.dubbo.rpc.support.LocalException;
-
-import com.alibaba.com.caucho.hessian.HessianException;
-import org.junit.jupiter.api.Assertions;
-import org.junit.jupiter.api.Test;
-import org.mockito.Mockito;
-
-import static org.junit.jupiter.api.Assertions.assertEquals;
-import static org.mockito.ArgumentMatchers.eq;
-import static org.mockito.BDDMockito.given;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-/**
- * ExceptionFilterTest
- */
-public class ExceptionFilterTest {
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testRpcException() {
-        Logger logger = mock(Logger.class);
-        RpcContext.getContext().setRemoteAddress("127.0.0.1", 1234);
-        RpcException exception = new RpcException("TestRpcException");
-
-        ExceptionFilter exceptionFilter = new ExceptionFilter();
-        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        given(invoker.getInterface()).willReturn(DemoService.class);
-        given(invoker.invoke(eq(invocation))).willThrow(exception);
-
-        try {
-            exceptionFilter.invoke(invoker, invocation);
-        } catch (RpcException e) {
-            assertEquals("TestRpcException", e.getMessage());
-            exceptionFilter.setLogger(logger);
-            exceptionFilter.onError(e, invoker, invocation);
-        }
-
-        Mockito.verify(logger).error(eq("Got unchecked and undeclared exception which called by 127.0.0.1. service: "
-                + DemoService.class.getName() + ", method: sayHello, exception: "
-                + RpcException.class.getName() + ": TestRpcException"), eq(exception));
-        RpcContext.removeContext();
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testJavaException() {
-
-        ExceptionFilter exceptionFilter = new ExceptionFilter();
-        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
-
-        AppResponse appResponse = new AppResponse();
-        appResponse.setException(new IllegalArgumentException("java"));
-
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        when(invoker.invoke(invocation)).thenReturn(appResponse);
-        when(invoker.getInterface()).thenReturn(DemoService.class);
-
-        Result newResult = exceptionFilter.invoke(invoker, invocation);
-
-        Assertions.assertEquals(appResponse.getException(), newResult.getException());
-
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testRuntimeException() {
-
-        ExceptionFilter exceptionFilter = new ExceptionFilter();
-        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
-
-        AppResponse appResponse = new AppResponse();
-        appResponse.setException(new LocalException("localException"));
-
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        when(invoker.invoke(invocation)).thenReturn(appResponse);
-        when(invoker.getInterface()).thenReturn(DemoService.class);
-
-        Result newResult = exceptionFilter.invoke(invoker, invocation);
-
-        Assertions.assertEquals(appResponse.getException(), newResult.getException());
-
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test
-    public void testConvertToRunTimeException() throws Exception {
-
-        ExceptionFilter exceptionFilter = new ExceptionFilter();
-        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
-
-        AppResponse mockRpcResult = new AppResponse();
-        mockRpcResult.setException(new HessianException("hessian"));
-        Result mockAsyncResult = AsyncRpcResult.newDefaultAsyncResult(mockRpcResult, invocation);
-
-
-        Invoker<DemoService> invoker = mock(Invoker.class);
-        when(invoker.invoke(invocation)).thenReturn(mockAsyncResult);
-        when(invoker.getInterface()).thenReturn(DemoService.class);
-
-        Result asyncResult = exceptionFilter.invoke(invoker, invocation);
-
-        AppResponse appResponse = (AppResponse) asyncResult.get();
-        exceptionFilter.onResponse(appResponse, invoker, invocation);
-
-        Assertions.assertFalse(appResponse.getException() instanceof HessianException);
-
-        Assertions.assertEquals(appResponse.getException().getClass(), RuntimeException.class);
-    }
-
+/*
+ * 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.rpc.filter;
+
+import org.apache.dubbo.common.logger.Logger;
+import org.apache.dubbo.rpc.AppResponse;
+import org.apache.dubbo.rpc.AsyncRpcResult;
+import org.apache.dubbo.rpc.Invoker;
+import org.apache.dubbo.rpc.Result;
+import org.apache.dubbo.rpc.RpcContext;
+import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.RpcInvocation;
+import org.apache.dubbo.rpc.support.DemoService;
+import org.apache.dubbo.rpc.support.LocalException;
+
+import com.alibaba.com.caucho.hessian.HessianException;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.mockito.Mockito;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.BDDMockito.given;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+/**
+ * ExceptionFilterTest
+ */
+public class ExceptionFilterTest {
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testRpcException() {
+        Logger logger = mock(Logger.class);
+        RpcContext.getContext().setRemoteAddress("127.0.0.1", 1234);
+        RpcException exception = new RpcException("TestRpcException");
+
+        ExceptionFilter exceptionFilter = new ExceptionFilter();
+        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        given(invoker.getInterface()).willReturn(DemoService.class);
+        given(invoker.invoke(eq(invocation))).willThrow(exception);
+
+        try {
+            exceptionFilter.invoke(invoker, invocation);
+        } catch (RpcException e) {
+            assertEquals("TestRpcException", e.getMessage());
+            exceptionFilter.setLogger(logger);
+            exceptionFilter.onError(e, invoker, invocation);
+        }
+
+        Mockito.verify(logger).error(eq("Got unchecked and undeclared exception which called by 127.0.0.1. service: "
+                + DemoService.class.getName() + ", method: sayHello, exception: "
+                + RpcException.class.getName() + ": TestRpcException"), eq(exception));
+        RpcContext.removeContext();
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testJavaException() {
+
+        ExceptionFilter exceptionFilter = new ExceptionFilter();
+        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
+
+        AppResponse appResponse = new AppResponse();
+        appResponse.setException(new IllegalArgumentException("java"));
+
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        when(invoker.invoke(invocation)).thenReturn(appResponse);
+        when(invoker.getInterface()).thenReturn(DemoService.class);
+
+        Result newResult = exceptionFilter.invoke(invoker, invocation);
+
+        Assertions.assertEquals(appResponse.getException(), newResult.getException());
+
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testRuntimeException() {
+
+        ExceptionFilter exceptionFilter = new ExceptionFilter();
+        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
+
+        AppResponse appResponse = new AppResponse();
+        appResponse.setException(new LocalException("localException"));
+
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        when(invoker.invoke(invocation)).thenReturn(appResponse);
+        when(invoker.getInterface()).thenReturn(DemoService.class);
+
+        Result newResult = exceptionFilter.invoke(invoker, invocation);
+
+        Assertions.assertEquals(appResponse.getException(), newResult.getException());
+
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test
+    public void testConvertToRunTimeException() throws Exception {
+
+        ExceptionFilter exceptionFilter = new ExceptionFilter();
+        RpcInvocation invocation = new RpcInvocation("sayHello", DemoService.class.getName(), "", new Class<?>[]{String.class}, new Object[]{"world"});
+
+        AppResponse mockRpcResult = new AppResponse();
+        mockRpcResult.setException(new HessianException("hessian"));
+        Result mockAsyncResult = AsyncRpcResult.newDefaultAsyncResult(mockRpcResult, invocation);
+
+
+        Invoker<DemoService> invoker = mock(Invoker.class);
+        when(invoker.invoke(invocation)).thenReturn(mockAsyncResult);
+        when(invoker.getInterface()).thenReturn(DemoService.class);
+
+        Result asyncResult = exceptionFilter.invoke(invoker, invocation);
+
+        AppResponse appResponse = (AppResponse) asyncResult.get();
+        exceptionFilter.onResponse(appResponse, invoker, invocation);
+
+        Assertions.assertFalse(appResponse.getException() instanceof HessianException);
+
+        Assertions.assertEquals(appResponse.getException().getClass(), RuntimeException.class);
+    }
+
 }
\ No newline at end of file