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/03/17 11:43:35 UTC

[dubbo] branch 3.0-preview updated: Refactor dubbo reference processing (#7383)

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

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


The following commit(s) were added to refs/heads/3.0-preview by this push:
     new fc73115  Refactor dubbo reference processing (#7383)
fc73115 is described below

commit fc731156ed2cf0e4b6a6d60d333782200dfdc897
Author: Gong Dewei <ky...@qq.com>
AuthorDate: Wed Mar 17 19:42:41 2021 +0800

    Refactor dubbo reference processing (#7383)
---
 .../org/apache/dubbo/config/AbstractConfig.java    |  10 +
 .../apache/dubbo/config/ReferenceConfigBase.java   |  50 ++-
 .../apache/dubbo/common/utils/NetUtilsTest.java    |   2 +-
 .../dubbo/config/bootstrap/DubboBootstrap.java     |   8 +
 .../DubboConfigInitializationPostProcessor.java    | 131 ++++++
 .../apache/dubbo/config/spring/ReferenceBean.java  | 226 +++++++---
 .../dubbo/config/spring/ReferenceBeanManager.java  | 244 ++++++++++
 .../AbstractAnnotationBeanPostProcessor.java       | 489 +++++++++++++++++++++
 .../AbstractAnnotationConfigBeanBuilder.java       | 215 ---------
 .../AnnotatedInterfaceConfigBeanBuilder.java       | 215 ---------
 .../AnnotationPropertyValuesAdapter.java           |   2 +-
 .../ReferenceAnnotationBeanPostProcessor.java      | 364 +++++++++------
 .../factory/annotation/ReferenceBeanBuilder.java   | 241 ++++++----
 .../annotation/ServiceClassPostProcessor.java      |  12 +-
 .../spring/schema/DubboBeanDefinitionParser.java   | 239 ++++++----
 .../dubbo/config/spring/util/DubboBeanUtils.java   | 103 ++++-
 .../org/apache/dubbo/config/spring/ConfigTest.java |  96 ++--
 .../AnnotationPropertyValuesAdapterTest.java       |  12 +-
 .../ReferenceAnnotationBeanPostProcessorTest.java  | 107 ++---
 .../annotation/ReferenceBeanBuilderTest.java       |  15 +-
 .../DubboComponentScanRegistrarTest.java           |   6 +-
 .../annotation/DubboConfigConfigurationTest.java   |   2 +
 .../context/annotation/EnableDubboConfigTest.java  |  12 +
 .../spring/context/annotation/EnableDubboTest.java |   8 +-
 .../dubbo/config/spring/issues/Issue6252Test.java  |  12 +-
 .../spring/schema/DubboNamespaceHandlerTest.java   |  34 +-
 .../config/spring/schema/GenericServiceTest.java   |  10 +-
 .../resources/META-INF/init-reference.properties   |   5 +
 .../apache/dubbo/config/spring/demo-provider.xml   |   8 +-
 ...reference.xml => init-reference-properties.xml} |  14 +-
 .../apache/dubbo/config/spring/init-reference.xml  |  20 +-
 .../dubbo/metadata/definition/util/ClassUtils.java |   4 +-
 .../registry/multicast/MulticastRegistryTest.java  |  11 +-
 .../dubbo/rpc/proxy/AbstractProxyFactory.java      |   4 +
 34 files changed, 1968 insertions(+), 963 deletions(-)

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 47556c5..3e0c9f6 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
@@ -161,6 +161,11 @@ public abstract class AbstractConfig implements Serializable {
         }
     }
 
+    /**
+     * Put attributes of specify 'config' into 'parameters' argument
+     * @param parameters
+     * @param config
+     */
     @Deprecated
     protected static void appendAttributes(Map<String, Object> parameters, Object config) {
         appendAttributes(parameters, config, null);
@@ -359,6 +364,11 @@ public abstract class AbstractConfig implements Serializable {
         }
     }
 
+    /**
+     * Copy attributes from annotation
+     * @param annotationClass
+     * @param annotation
+     */
     protected void appendAnnotation(Class<?> annotationClass, Object annotation) {
         Method[] methods = annotationClass.getMethods();
         for (Method method : methods) {
diff --git a/dubbo-common/src/main/java/org/apache/dubbo/config/ReferenceConfigBase.java b/dubbo-common/src/main/java/org/apache/dubbo/config/ReferenceConfigBase.java
index 966b384..3a149aa 100644
--- a/dubbo-common/src/main/java/org/apache/dubbo/config/ReferenceConfigBase.java
+++ b/dubbo-common/src/main/java/org/apache/dubbo/config/ReferenceConfigBase.java
@@ -119,6 +119,11 @@ public abstract class ReferenceConfigBase<T> extends AbstractReferenceConfig {
         }
     }
 
+    /**
+     * Get actual interface class of this reference.
+     * The actual service type of remote provider.
+     * @return
+     */
     public Class<?> getActualInterface() {
         Class actualInterface = interfaceClass;
         if (interfaceClass == GenericService.class) {
@@ -131,33 +136,43 @@ public abstract class ReferenceConfigBase<T> extends AbstractReferenceConfig {
         return actualInterface;
     }
 
+    /**
+     * Get proxy interface class of this reference.
+     * The proxy interface class is used to create proxy instance.
+     * @return
+     */
     public Class<?> getInterfaceClass() {
         if (interfaceClass != null) {
             return interfaceClass;
         }
-        if (ProtocolUtils.isGeneric(getGeneric())
-                || (getConsumer() != null && ProtocolUtils.isGeneric(getConsumer().getGeneric()))) {
+
+        String generic = getGeneric();
+        if (StringUtils.isBlank(generic) && getConsumer() != null) {
+            generic = getConsumer().getGeneric();
+        }
+        interfaceClass = determineInterfaceClass(generic, interfaceName);
+
+        return interfaceClass;
+    }
+
+    /**
+     * Determine the interface of the proxy class
+     * @param generic
+     * @param interfaceName
+     * @return
+     */
+    public static Class<?> determineInterfaceClass(String generic, String interfaceName) {
+        if (ProtocolUtils.isGeneric(generic)) {
             return GenericService.class;
         }
         try {
             if (interfaceName != null && interfaceName.length() > 0) {
-                interfaceClass = Class.forName(interfaceName, true, ClassUtils.getClassLoader());
+                return Class.forName(interfaceName, true, ClassUtils.getClassLoader());
             }
         } catch (ClassNotFoundException t) {
             throw new IllegalStateException(t.getMessage(), t);
         }
-
-        return interfaceClass;
-    }
-
-    /**
-     * @param interfaceClass
-     * @see #setInterface(Class)
-     * @deprecated
-     */
-    @Deprecated
-    public void setInterfaceClass(Class<?> interfaceClass) {
-        setInterface(interfaceClass);
+        return null;
     }
 
     public String getInterface() {
@@ -166,17 +181,12 @@ public abstract class ReferenceConfigBase<T> extends AbstractReferenceConfig {
 
     public void setInterface(String interfaceName) {
         this.interfaceName = interfaceName;
-        // FIXME, add id strategy in ConfigManager
-//        if (StringUtils.isEmpty(id)) {
-//            id = interfaceName;
-//        }
     }
 
     public void setInterface(Class<?> interfaceClass) {
         if (interfaceClass != null && !interfaceClass.isInterface()) {
             throw new IllegalStateException("The interface class " + interfaceClass + " is not a interface!");
         }
-        this.interfaceClass = interfaceClass;
         setInterface(interfaceClass == null ? null : interfaceClass.getName());
     }
 
diff --git a/dubbo-common/src/test/java/org/apache/dubbo/common/utils/NetUtilsTest.java b/dubbo-common/src/test/java/org/apache/dubbo/common/utils/NetUtilsTest.java
index c4f51d7..ea2c54e 100644
--- a/dubbo-common/src/test/java/org/apache/dubbo/common/utils/NetUtilsTest.java
+++ b/dubbo-common/src/test/java/org/apache/dubbo/common/utils/NetUtilsTest.java
@@ -162,7 +162,7 @@ public class NetUtilsTest {
     @Test
     public void testGetIpByHost() throws Exception {
         assertThat(NetUtils.getIpByHost("localhost"), equalTo("127.0.0.1"));
-        assertThat(NetUtils.getIpByHost("dubbo"), equalTo("dubbo"));
+        assertThat(NetUtils.getIpByHost("dubbo.local"), equalTo("dubbo.local"));
     }
 
     @Test
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 cdf66dd..0ef8ef4 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
@@ -202,6 +202,14 @@ public class DubboBootstrap extends GenericEventListener {
         return instance;
     }
 
+    public static void reset() {
+        if (instance != null) {
+            instance.destroy();
+        }
+        ApplicationModel.reset();
+        instance = null;
+    }
+
     private DubboBootstrap() {
         configManager = ApplicationModel.getConfigManager();
         environment = ApplicationModel.getEnvironment();
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/DubboConfigInitializationPostProcessor.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/DubboConfigInitializationPostProcessor.java
new file mode 100644
index 0000000..068d700
--- /dev/null
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/DubboConfigInitializationPostProcessor.java
@@ -0,0 +1,131 @@
+/*
+ * 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.config.spring;
+
+import org.apache.dubbo.config.ApplicationConfig;
+import org.apache.dubbo.config.ConsumerConfig;
+import org.apache.dubbo.config.MetadataReportConfig;
+import org.apache.dubbo.config.MetricsConfig;
+import org.apache.dubbo.config.ModuleConfig;
+import org.apache.dubbo.config.MonitorConfig;
+import org.apache.dubbo.config.ProtocolConfig;
+import org.apache.dubbo.config.ProviderConfig;
+import org.apache.dubbo.config.RegistryConfig;
+import org.apache.dubbo.config.SslConfig;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.FatalBeanException;
+import org.springframework.beans.factory.BeanFactory;
+import org.springframework.beans.factory.BeanFactoryAware;
+import org.springframework.beans.factory.BeanInitializationException;
+import org.springframework.beans.factory.annotation.Value;
+import org.springframework.beans.factory.config.BeanPostProcessor;
+import org.springframework.beans.factory.config.ConfigurableListableBeanFactory;
+import org.springframework.core.Ordered;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.springframework.beans.factory.BeanFactoryUtils.beansOfTypeIncludingAncestors;
+
+/**
+ *
+ * Post-processor Dubbo config bean initialization.
+ *
+ * NOTE: Dubbo config beans MUST be initialized after registering all BeanPostProcessors,
+ * that is after the AbstractApplicationContext#registerBeanPostProcessors() method.
+ */
+public class DubboConfigInitializationPostProcessor implements BeanPostProcessor, BeanFactoryAware, Ordered {
+
+    public static String BEAN_NAME = "dubboBeanFactoryPostProcessor";
+
+    /**
+     * This bean post processor should run before seata GlobalTransactionScanner(1024)
+     */
+    @Value("${dubbo.config-initialization-post-processor.order:1000}")
+    private int order = 1000;
+
+    private AtomicBoolean initialized = new AtomicBoolean(false);
+    private ConfigurableListableBeanFactory beanFactory;
+    private ReferenceBeanManager referenceBeanManager;
+
+    @Override
+    public Object postProcessBeforeInitialization(Object bean, String beanName) throws BeansException {
+        if (initialized.compareAndSet(false, true)) {
+            try {
+                prepareDubboConfigBeans(beanFactory);
+                prepareReferenceBeans(beanFactory);
+            } catch (Throwable e) {
+                throw new FatalBeanException("Initialization dubbo config beans failed", e);
+            }
+        }
+        return bean;
+    }
+
+    @Override
+    public Object postProcessAfterInitialization(Object bean, String beanName) throws BeansException {
+        try {
+            if (bean instanceof ReferenceBean) {
+                ReferenceBean referenceBean = (ReferenceBean) bean;
+                referenceBeanManager.addReference(referenceBean);
+            }
+        } catch (Exception e) {
+            throw new BeanInitializationException("Initialization reference bean failed", e);
+        }
+        return bean;
+    }
+
+    @Override
+    public void setBeanFactory(BeanFactory beanFactory) throws BeansException {
+        this.beanFactory = (ConfigurableListableBeanFactory) beanFactory;
+        referenceBeanManager = beanFactory.getBean(ReferenceBeanManager.BEAN_NAME, ReferenceBeanManager.class);
+    }
+
+    @Override
+    public int getOrder() {
+        return order;
+    }
+
+    public void setOrder(int order) {
+        this.order = order;
+    }
+
+    private void prepareReferenceBeans(ConfigurableListableBeanFactory beanFactory) throws Exception {
+        ReferenceBeanManager referenceBeanManager = beanFactory.getBean(ReferenceBeanManager.BEAN_NAME, ReferenceBeanManager.class);
+        referenceBeanManager.prepareReferenceBeans();
+    }
+
+    /**
+     * Initializes there Dubbo's Config Beans before @Reference bean autowiring
+     */
+    private void prepareDubboConfigBeans(ConfigurableListableBeanFactory beanFactory) {
+        //Make sure all these config beans are inited and registered to ConfigManager
+        beansOfTypeIncludingAncestors(beanFactory, ApplicationConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, ModuleConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, RegistryConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, ProtocolConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, MonitorConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, ProviderConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, ConsumerConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, ConfigCenterBean.class);
+        beansOfTypeIncludingAncestors(beanFactory, MetadataReportConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, MetricsConfig.class);
+        beansOfTypeIncludingAncestors(beanFactory, SslConfig.class);
+
+        //SHOULD NOT init service beans here, avoid conflicts with seata
+        //beansOfTypeIncludingAncestors(beanFactory, ServiceBean.class);
+    }
+
+}
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBean.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBean.java
index be9cd71..f8f834b 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBean.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBean.java
@@ -16,56 +16,79 @@
  */
 package org.apache.dubbo.config.spring;
 
-import org.apache.dubbo.config.ApplicationConfig;
-import org.apache.dubbo.config.ConsumerConfig;
-import org.apache.dubbo.config.MetadataReportConfig;
-import org.apache.dubbo.config.MetricsConfig;
-import org.apache.dubbo.config.ModuleConfig;
-import org.apache.dubbo.config.MonitorConfig;
-import org.apache.dubbo.config.ProtocolConfig;
-import org.apache.dubbo.config.ProviderConfig;
+import org.apache.dubbo.common.utils.Assert;
+import org.apache.dubbo.common.utils.ReflectUtils;
+import org.apache.dubbo.common.utils.StringUtils;
 import org.apache.dubbo.config.ReferenceConfig;
-import org.apache.dubbo.config.RegistryConfig;
-import org.apache.dubbo.config.SslConfig;
-import org.apache.dubbo.config.annotation.Reference;
-import org.apache.dubbo.config.spring.extension.SpringExtensionFactory;
 import org.apache.dubbo.config.support.Parameter;
-
+import org.apache.dubbo.config.utils.ReferenceConfigCache;
+import org.apache.dubbo.rpc.proxy.AbstractProxyFactory;
+import org.apache.dubbo.rpc.support.ProtocolUtils;
+import org.springframework.aop.framework.ProxyFactory;
+import org.springframework.aop.target.AbstractLazyCreationTargetSource;
+import org.springframework.beans.MutablePropertyValues;
+import org.springframework.beans.factory.BeanClassLoaderAware;
 import org.springframework.beans.factory.DisposableBean;
 import org.springframework.beans.factory.FactoryBean;
 import org.springframework.beans.factory.InitializingBean;
+import org.springframework.beans.factory.config.BeanDefinition;
+import org.springframework.beans.factory.config.ConfigurableListableBeanFactory;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationContextAware;
 
-import static org.springframework.beans.factory.BeanFactoryUtils.beansOfTypeIncludingAncestors;
+import java.util.Map;
+
 
 /**
  * ReferenceFactoryBean
  */
-public class ReferenceBean<T> extends ReferenceConfig<T> implements FactoryBean,
-        ApplicationContextAware, InitializingBean, DisposableBean {
-
-    private static final long serialVersionUID = 213195494150089726L;
+public class ReferenceBean<T> implements FactoryBean,
+        ApplicationContextAware, BeanClassLoaderAware, InitializingBean, DisposableBean {
 
     private transient ApplicationContext applicationContext;
+    private ClassLoader beanClassLoader;
+    private DubboReferenceLazyInitTargetSource referenceTargetSource;
+    private Object referenceLazyProxy;
+    /**
+     * The interface class of the reference service
+     */
+    protected Class<?> interfaceClass;
+
+    //beanName
+    protected String id;
+    //from annotation attributes
+    private Map<String, Object> referenceProps;
+    //from bean definition
+    private MutablePropertyValues propertyValues;
+    //actual reference config
+    private ReferenceConfig referenceConfig;
+    private String generic;
+    private String interfaceName;
 
     public ReferenceBean() {
         super();
     }
 
-    public ReferenceBean(Reference reference) {
-        super(reference);
+    public ReferenceBean(Map<String, Object> referenceProps) {
+        this.referenceProps = referenceProps;
     }
 
     @Override
     public void setApplicationContext(ApplicationContext applicationContext) {
         this.applicationContext = applicationContext;
-        SpringExtensionFactory.addApplicationContext(applicationContext);
+    }
+
+    @Override
+    public void setBeanClassLoader(ClassLoader classLoader) {
+        this.beanClassLoader = classLoader;
     }
 
     @Override
     public Object getObject() {
-        return get();
+        if (referenceLazyProxy == null) {
+            createReferenceLazyProxy();
+        }
+        return referenceLazyProxy;
     }
 
     @Override
@@ -79,43 +102,148 @@ public class ReferenceBean<T> extends ReferenceConfig<T> implements FactoryBean,
         return true;
     }
 
+    @Override
+    public void afterPropertiesSet() throws Exception {
+        if (referenceProps == null) {
+            Assert.notEmptyString(getId(), "The id of ReferenceBean cannot be empty");
+            ConfigurableListableBeanFactory beanFactory = getBeanFactory();
+            BeanDefinition beanDefinition = beanFactory.getMergedBeanDefinition(getId());
+            propertyValues = beanDefinition.getPropertyValues();
+        }
+    }
+
+    private ConfigurableListableBeanFactory getBeanFactory() {
+        return (ConfigurableListableBeanFactory) applicationContext.getAutowireCapableBeanFactory();
+    }
+
+    @Override
+    public void destroy() {
+        // do nothing
+    }
+
     /**
-     * Initializes there Dubbo's Config Beans before @Reference bean autowiring
+     * TODO remove get() method
+     *
+     * @return
      */
-    private void prepareDubboConfigBeans() {
-        beansOfTypeIncludingAncestors(applicationContext, ApplicationConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, ModuleConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, RegistryConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, ProtocolConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, MonitorConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, ProviderConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, ConsumerConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, ConfigCenterBean.class);
-        beansOfTypeIncludingAncestors(applicationContext, MetadataReportConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, MetricsConfig.class);
-        beansOfTypeIncludingAncestors(applicationContext, SslConfig.class);
+    @Deprecated
+    public Object get() {
+        throw new UnsupportedOperationException("Should not call this method");
     }
 
-    @Override
-    @SuppressWarnings({"unchecked"})
-    public void afterPropertiesSet() throws Exception {
+    public String getId() {
+        return id;
+    }
+
+    public void setId(String id) {
+        this.id = id;
+    }
 
-        // Initializes Dubbo's Config Beans before @Reference bean autowiring
-        prepareDubboConfigBeans();
+    /* Compatible with seata: io.seata.rm.tcc.remoting.parser.DubboRemotingParser#getServiceDesc() */
+    public String getGroup() {
+        return referenceConfig.getGroup();
+    }
+
+    public String getVersion() {
+        return referenceConfig.getVersion();
+    }
+
+    public Map<String, Object> getReferenceProps() {
+        return referenceProps;
+    }
+
+    public MutablePropertyValues getPropertyValues() {
+        return propertyValues;
+    }
+
+    public ReferenceConfig getReferenceConfig() {
+        return referenceConfig;
+    }
+
+    public void setReferenceConfig(ReferenceConfig referenceConfig) {
+        this.referenceConfig = referenceConfig;
+    }
+
+    public Class<?> getInterfaceClass() {
+        // get interface class
+        if (interfaceClass == null) {
+            if (referenceProps != null) {
+                //get interface class name of @DubboReference
+                String interfaceName = (String) referenceProps.get("interfaceName");
+                if (interfaceName == null) {
+                    Class clazz = (Class) referenceProps.get("interfaceClass");
+                    if (clazz != null) {
+                        interfaceName = clazz.getName();
+                    }
+                }
+                if (StringUtils.isBlank(interfaceName)) {
+                    throw new RuntimeException("Need to specify the 'interfaceName' or 'interfaceClass' attribute of '@DubboReference'");
+                }
+                this.interfaceName = interfaceName;
+
+                //get generic
+                Object genericValue = referenceProps.get("generic");
+                generic = genericValue != null ? genericValue.toString() : null;
+                String consumer = (String) referenceProps.get("consumer");
+                if (StringUtils.isBlank(generic) && consumer != null) {
+                    // get generic from consumerConfig
+                    BeanDefinition consumerBeanDefinition = getBeanFactory().getMergedBeanDefinition(consumer);
+                    if (consumerBeanDefinition != null) {
+                        generic = (String) consumerBeanDefinition.getPropertyValues().get("generic");
+                    }
+                }
+            } else if (propertyValues != null) {
+                generic = (String) propertyValues.get("generic");
+                interfaceName = (String) propertyValues.get("interface");
+            } else {
+                throw new RuntimeException("Required 'referenceProps' or beanDefinition");
+            }
+
+            interfaceClass = ReferenceConfig.determineInterfaceClass(generic, interfaceName);
+        }
+        return interfaceClass;
+    }
 
-        // lazy init by default.
-        if (init == null) {
-            init = false;
+    private void createReferenceLazyProxy() {
+        this.referenceTargetSource = new DubboReferenceLazyInitTargetSource();
+
+        //set proxy interfaces
+        //see also: org.apache.dubbo.rpc.proxy.AbstractProxyFactory.getProxy(org.apache.dubbo.rpc.Invoker<T>, boolean)
+        ProxyFactory proxyFactory = new ProxyFactory();
+        proxyFactory.setTargetSource(referenceTargetSource);
+        proxyFactory.addInterface(getInterfaceClass());
+        Class<?>[] internalInterfaces = AbstractProxyFactory.getInternalInterfaces();
+        for (Class<?> anInterface : internalInterfaces) {
+            proxyFactory.addInterface(anInterface);
+        }
+        if (ProtocolUtils.isGeneric(generic)){
+            //add actual interface
+            proxyFactory.addInterface(ReflectUtils.forName(interfaceName));
         }
 
-        // eager init if necessary.
-        if (shouldInit()) {
-            getObject();
+        this.referenceLazyProxy = proxyFactory.getProxy(this.beanClassLoader);
+    }
+
+    private Object getCallProxy() throws Exception {
+
+        if (referenceConfig == null) {
+            throw new IllegalStateException("ReferenceBean is not ready yet, maybe dubbo engine is not started");
         }
+        //get reference proxy
+        return ReferenceConfigCache.getCache().get(referenceConfig);
     }
 
-    @Override
-    public void destroy() {
-        // do nothing
+    private class DubboReferenceLazyInitTargetSource extends AbstractLazyCreationTargetSource {
+
+        @Override
+        protected Object createObject() throws Exception {
+            return getCallProxy();
+        }
+
+        @Override
+        public synchronized Class<?> getTargetClass() {
+            return getInterfaceClass();
+        }
     }
+
 }
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBeanManager.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBeanManager.java
new file mode 100644
index 0000000..122440c
--- /dev/null
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/ReferenceBeanManager.java
@@ -0,0 +1,244 @@
+/*
+ * 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.config.spring;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.dubbo.common.utils.Assert;
+import org.apache.dubbo.config.ArgumentConfig;
+import org.apache.dubbo.config.MethodConfig;
+import org.apache.dubbo.config.ReferenceConfig;
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
+import org.apache.dubbo.config.spring.beans.factory.annotation.AnnotationPropertyValuesAdapter;
+import org.apache.dubbo.config.spring.beans.factory.annotation.ReferenceBeanBuilder;
+import org.apache.dubbo.config.utils.ReferenceConfigCache;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.MutablePropertyValues;
+import org.springframework.beans.PropertyValue;
+import org.springframework.beans.factory.config.BeanDefinition;
+import org.springframework.beans.factory.config.BeanDefinitionHolder;
+import org.springframework.beans.factory.config.RuntimeBeanReference;
+import org.springframework.beans.factory.config.TypedStringValue;
+import org.springframework.beans.factory.support.ManagedList;
+import org.springframework.beans.factory.support.ManagedMap;
+import org.springframework.context.ApplicationContext;
+import org.springframework.context.ApplicationContextAware;
+import org.springframework.core.annotation.AnnotationAttributes;
+import org.springframework.core.env.Environment;
+import org.springframework.core.env.PropertyResolver;
+import org.springframework.validation.DataBinder;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+
+public class ReferenceBeanManager implements ApplicationContextAware {
+    public static final String BEAN_NAME = "dubboReferenceBeanManager";
+    private final Log logger = LogFactory.getLog(getClass());
+    private Map<String, ReferenceBean> configMap = new ConcurrentHashMap<>();
+    private ApplicationContext applicationContext;
+    private volatile boolean initialized = false;
+
+
+    public void addReference(ReferenceBean referenceBean) throws Exception {
+        Assert.notNull(referenceBean.getId(), "The id of ReferenceBean cannot be empty");
+        //TODO generate reference bean id and unique cache key
+        String key = referenceBean.getId();
+        ReferenceBean oldReferenceBean = configMap.get(key);
+        if (oldReferenceBean != null) {
+            if (referenceBean != oldReferenceBean) {
+                logger.warn("Found duplicated ReferenceBean id: " + key);
+            }
+            return;
+        }
+        configMap.put(key, referenceBean);
+
+        // if add reference after prepareReferenceBeans(), should init it immediately.
+        if (initialized) {
+            initReferenceBean(referenceBean, true);
+        }
+    }
+
+    public ReferenceBean get(String id) {
+        return configMap.get(id);
+    }
+
+//    public ReferenceBean getOrCreateReference(Map<String, String> referenceProps) {
+//        Integer key = referenceProps.hashCode();
+//        return configMap.computeIfAbsent(key, k -> {
+//            ReferenceBean referenceBean = new ReferenceBean();
+//            referenceBean.setReferenceProps(referenceProps);
+//            //referenceBean.setId();
+//            return referenceBean;
+//        });
+//    }
+
+    public Collection<ReferenceBean> getReferences() {
+        return configMap.values();
+    }
+
+    @Override
+    public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
+        this.applicationContext = applicationContext;
+    }
+
+    /**
+     * Initialize all reference beans, call at Dubbo starting
+     * @throws Exception
+     */
+    public void prepareReferenceBeans() throws Exception {
+        initialized = true;
+        for (ReferenceBean referenceBean : getReferences()) {
+            initReferenceBean(referenceBean, false);
+        }
+    }
+
+    /**
+     * NOTE: This method should only call after all dubbo config beans and all property resolvers is loaded.
+     *
+     * @param referenceBean
+     * @throws Exception
+     */
+    private void initReferenceBean(ReferenceBean referenceBean, boolean canCreateProxy) throws Exception {
+
+        if (referenceBean.getReferenceConfig() != null) {
+            return;
+        }
+
+        Environment environment = applicationContext.getEnvironment();
+        Map<String, Object> referenceProps = referenceBean.getReferenceProps();
+        if (referenceProps == null) {
+            MutablePropertyValues propertyValues = referenceBean.getPropertyValues();
+            if (propertyValues == null) {
+                throw new RuntimeException("ReferenceBean is invalid, missing 'propertyValues'");
+            }
+            referenceProps = toReferenceProps(propertyValues, environment);
+        }
+
+        //resolve placeholders
+        resolvePlaceholders(referenceProps, environment);
+
+        //create real ReferenceConfig
+        ReferenceConfig referenceConfig = ReferenceBeanBuilder.create(new AnnotationAttributes(new LinkedHashMap<>(referenceProps)), applicationContext)
+                .defaultInterfaceClass(referenceBean.getObjectType())
+                .build();
+
+        referenceBean.setReferenceConfig(referenceConfig);
+
+        // register ReferenceConfig
+        DubboBootstrap.getInstance().reference(referenceConfig);
+
+        //TODO add after DubboBootstrap is started
+        if (canCreateProxy && referenceConfig.shouldInit()) {
+            ReferenceConfigCache.getCache().get(referenceConfig);
+        }
+    }
+
+    private void resolvePlaceholders(Map<String, Object> referenceProps, PropertyResolver propertyResolver) {
+        for (Map.Entry<String, Object> entry : referenceProps.entrySet()) {
+            Object value = entry.getValue();
+            if (value instanceof String) {
+                String valueToResovle = (String) value;
+                entry.setValue(propertyResolver.resolveRequiredPlaceholders(valueToResovle));
+            } else if (value instanceof String[]) {
+                String[] strings = (String[]) value;
+                for (int i = 0; i < strings.length; i++) {
+                    strings[i] = propertyResolver.resolveRequiredPlaceholders(strings[i]);
+                }
+                entry.setValue(strings);
+            }
+        }
+    }
+
+    private Map<String, Object> toReferenceProps(MutablePropertyValues propertyValues, PropertyResolver propertyResolver) {
+        Map<String, Object> referenceProps;
+        referenceProps = new LinkedHashMap<>();
+        for (PropertyValue propertyValue : propertyValues.getPropertyValueList()) {
+            String propertyName = propertyValue.getName();
+            Object value = propertyValue.getValue();
+            if ("methods".equals(propertyName)) {
+                ManagedList managedList = (ManagedList) value;
+                List<MethodConfig> methodConfigs = new ArrayList<>();
+                for (Object el : managedList) {
+                    MethodConfig methodConfig = createMethodConfig(((BeanDefinitionHolder) el).getBeanDefinition(), propertyResolver);
+                    methodConfigs.add(methodConfig);
+                }
+                value = methodConfigs.toArray(new MethodConfig[0]);
+            } else if ("parameters".equals(propertyName)) {
+                value = createParameterMap((ManagedMap) value, propertyResolver);
+            }
+            if (value instanceof RuntimeBeanReference) {
+                RuntimeBeanReference beanReference = (RuntimeBeanReference) value;
+                value = applicationContext.getBean(beanReference.getBeanName());
+            }
+            referenceProps.put(propertyName, value);
+        }
+        return referenceProps;
+    }
+
+    private MethodConfig createMethodConfig(BeanDefinition beanDefinition, PropertyResolver propertyResolver) {
+        Map<String, Object> attributes = new LinkedHashMap<>();
+        MutablePropertyValues pvs = beanDefinition.getPropertyValues();
+        for (PropertyValue propertyValue : pvs.getPropertyValueList()) {
+            String propertyName = propertyValue.getName();
+            Object value = propertyValue.getValue();
+            if ("arguments".equals(propertyName)) {
+                ManagedList managedList = (ManagedList) value;
+                List<ArgumentConfig> argumentConfigs = new ArrayList<>();
+                for (Object el : managedList) {
+                    ArgumentConfig argumentConfig = createArgumentConfig(((BeanDefinitionHolder) el).getBeanDefinition(), propertyResolver);
+                    argumentConfigs.add(argumentConfig);
+                }
+                value = argumentConfigs.toArray(new ArgumentConfig[0]);
+            } else if ("parameters".equals(propertyName)) {
+                value = createParameterMap((ManagedMap) value, propertyResolver);
+            }
+
+            if (value instanceof RuntimeBeanReference) {
+                RuntimeBeanReference beanReference = (RuntimeBeanReference) value;
+                value = applicationContext.getBean(beanReference.getBeanName());
+            }
+            attributes.put(propertyName, value);
+        }
+        MethodConfig methodConfig = new MethodConfig();
+        DataBinder dataBinder = new DataBinder(methodConfig);
+        dataBinder.bind(new AnnotationPropertyValuesAdapter(attributes, propertyResolver));
+        return methodConfig;
+    }
+
+    private ArgumentConfig createArgumentConfig(BeanDefinition beanDefinition, PropertyResolver propertyResolver) {
+        ArgumentConfig argumentConfig = new ArgumentConfig();
+        DataBinder dataBinder = new DataBinder(argumentConfig);
+        dataBinder.bind(beanDefinition.getPropertyValues());
+        return argumentConfig;
+    }
+
+    private Map<String, String> createParameterMap(ManagedMap managedMap, PropertyResolver propertyResolver) {
+        Map<String, String> map = new LinkedHashMap<>();
+        Set<Map.Entry<String, TypedStringValue>> entrySet = managedMap.entrySet();
+        for (Map.Entry<String, TypedStringValue> entry : entrySet) {
+            map.put(entry.getKey(), entry.getValue().getValue());
+        }
+        return map;
+    }
+
+
+}
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationBeanPostProcessor.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationBeanPostProcessor.java
new file mode 100644
index 0000000..2581b1e
--- /dev/null
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationBeanPostProcessor.java
@@ -0,0 +1,489 @@
+/*
+ * 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.config.spring.beans.factory.annotation;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.springframework.beans.BeanUtils;
+import org.springframework.beans.BeansException;
+import org.springframework.beans.PropertyValues;
+import org.springframework.beans.factory.BeanClassLoaderAware;
+import org.springframework.beans.factory.BeanCreationException;
+import org.springframework.beans.factory.BeanFactory;
+import org.springframework.beans.factory.BeanFactoryAware;
+import org.springframework.beans.factory.DisposableBean;
+import org.springframework.beans.factory.annotation.AutowiredAnnotationBeanPostProcessor;
+import org.springframework.beans.factory.annotation.InjectionMetadata;
+import org.springframework.beans.factory.config.BeanPostProcessor;
+import org.springframework.beans.factory.config.ConfigurableListableBeanFactory;
+import org.springframework.beans.factory.config.InstantiationAwareBeanPostProcessorAdapter;
+import org.springframework.beans.factory.support.MergedBeanDefinitionPostProcessor;
+import org.springframework.beans.factory.support.RootBeanDefinition;
+import org.springframework.context.EnvironmentAware;
+import org.springframework.core.Ordered;
+import org.springframework.core.PriorityOrdered;
+import org.springframework.core.annotation.AnnotationAttributes;
+import org.springframework.core.env.Environment;
+import org.springframework.util.Assert;
+import org.springframework.util.ClassUtils;
+import org.springframework.util.ReflectionUtils;
+import org.springframework.util.StringUtils;
+
+import java.beans.PropertyDescriptor;
+import java.lang.annotation.Annotation;
+import java.lang.reflect.Field;
+import java.lang.reflect.Member;
+import java.lang.reflect.Method;
+import java.lang.reflect.Modifier;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static com.alibaba.spring.util.AnnotationUtils.getAnnotationAttributes;
+import static org.springframework.core.BridgeMethodResolver.findBridgedMethod;
+import static org.springframework.core.BridgeMethodResolver.isVisibilityBridgeMethodPair;
+
+/**
+ * Abstract common {@link BeanPostProcessor} implementation for customized annotation that annotated injected-object.
+ */
+@SuppressWarnings("unchecked")
+public abstract class AbstractAnnotationBeanPostProcessor extends
+        InstantiationAwareBeanPostProcessorAdapter implements MergedBeanDefinitionPostProcessor, PriorityOrdered,
+        BeanFactoryAware, BeanClassLoaderAware, EnvironmentAware, DisposableBean {
+
+    private final static int CACHE_SIZE = Integer.getInteger("", 32);
+
+    private final Log logger = LogFactory.getLog(getClass());
+
+    private final Class<? extends Annotation>[] annotationTypes;
+
+    private final ConcurrentMap<String, AbstractAnnotationBeanPostProcessor.AnnotatedInjectionMetadata> injectionMetadataCache =
+            new ConcurrentHashMap<String, AbstractAnnotationBeanPostProcessor.AnnotatedInjectionMetadata>(CACHE_SIZE);
+
+    private final ConcurrentMap<String, Object> injectedObjectsCache = new ConcurrentHashMap<String, Object>(CACHE_SIZE);
+
+    private ConfigurableListableBeanFactory beanFactory;
+
+    private Environment environment;
+
+    private ClassLoader classLoader;
+
+    /**
+     * make sure higher priority than {@link AutowiredAnnotationBeanPostProcessor}
+     */
+    private int order = Ordered.LOWEST_PRECEDENCE - 3;
+
+    /**
+     * @param annotationTypes the multiple types of {@link Annotation annotations}
+     */
+    public AbstractAnnotationBeanPostProcessor(Class<? extends Annotation>... annotationTypes) {
+        Assert.notEmpty(annotationTypes, "The argument of annotations' types must not empty");
+        this.annotationTypes = annotationTypes;
+    }
+
+    private static <T> Collection<T> combine(Collection<? extends T>... elements) {
+        List<T> allElements = new ArrayList<T>();
+        for (Collection<? extends T> e : elements) {
+            allElements.addAll(e);
+        }
+        return allElements;
+    }
+
+    /**
+     * Annotation type
+     *
+     * @return non-null
+     * @deprecated 2.7.3, uses {@link #getAnnotationTypes()}
+     */
+    @Deprecated
+    public final Class<? extends Annotation> getAnnotationType() {
+        return annotationTypes[0];
+    }
+
+    protected final Class<? extends Annotation>[] getAnnotationTypes() {
+        return annotationTypes;
+    }
+
+    @Override
+    public void setBeanFactory(BeanFactory beanFactory) throws BeansException {
+        Assert.isInstanceOf(ConfigurableListableBeanFactory.class, beanFactory,
+                "AnnotationInjectedBeanPostProcessor requires a ConfigurableListableBeanFactory");
+        this.beanFactory = (ConfigurableListableBeanFactory) beanFactory;
+    }
+
+    @Override
+    public void postProcessMergedBeanDefinition(RootBeanDefinition beanDefinition, Class<?> beanType, String beanName) {
+        if (beanType != null) {
+            AnnotatedInjectionMetadata metadata = findInjectionMetadata(beanName, beanType, null);
+            metadata.checkConfigMembers(beanDefinition);
+            try {
+                prepareInjection(metadata);
+            } catch (Exception e) {
+                logger.warn("Prepare injection of @"+getAnnotationType().getSimpleName()+" failed", e);
+            }
+        }
+    }
+
+    @Override
+    public PropertyValues postProcessPropertyValues(
+            PropertyValues pvs, PropertyDescriptor[] pds, Object bean, String beanName) throws BeanCreationException {
+
+        try {
+            AnnotatedInjectionMetadata metadata = findInjectionMetadata(beanName, bean.getClass(), pvs);
+            prepareInjection(metadata);
+            metadata.inject(bean, beanName, pvs);
+        } catch (BeanCreationException ex) {
+            throw ex;
+        } catch (Throwable ex) {
+            throw new BeanCreationException(beanName, "Injection of @" + getAnnotationType().getSimpleName()
+                    + " dependencies is failed", ex);
+        }
+        return pvs;
+    }
+
+
+    /**
+     * Finds {@link InjectionMetadata.InjectedElement} Metadata from annotated fields
+     *
+     * @param beanClass The {@link Class} of Bean
+     * @return non-null {@link List}
+     */
+    private List<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> findFieldAnnotationMetadata(final Class<?> beanClass) {
+
+        final List<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> elements = new LinkedList<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement>();
+
+        ReflectionUtils.doWithFields(beanClass, new ReflectionUtils.FieldCallback() {
+            @Override
+            public void doWith(Field field) throws IllegalArgumentException, IllegalAccessException {
+
+                for (Class<? extends Annotation> annotationType : getAnnotationTypes()) {
+
+                    AnnotationAttributes attributes = getAnnotationAttributes(field, annotationType, getEnvironment(), true, true);
+
+                    if (attributes != null) {
+
+                        if (Modifier.isStatic(field.getModifiers())) {
+                            if (logger.isWarnEnabled()) {
+                                logger.warn("@" + annotationType.getName() + " is not supported on static fields: " + field);
+                            }
+                            return;
+                        }
+
+                        elements.add(new AnnotatedFieldElement(field, attributes));
+                    }
+                }
+            }
+        });
+
+        return elements;
+
+    }
+
+    /**
+     * Finds {@link InjectionMetadata.InjectedElement} Metadata from annotated methods
+     *
+     * @param beanClass The {@link Class} of Bean
+     * @return non-null {@link List}
+     */
+    private List<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> findAnnotatedMethodMetadata(final Class<?> beanClass) {
+
+        final List<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> elements = new LinkedList<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement>();
+
+        ReflectionUtils.doWithMethods(beanClass, new ReflectionUtils.MethodCallback() {
+            @Override
+            public void doWith(Method method) throws IllegalArgumentException, IllegalAccessException {
+
+                Method bridgedMethod = findBridgedMethod(method);
+
+                if (!isVisibilityBridgeMethodPair(method, bridgedMethod)) {
+                    return;
+                }
+
+
+                for (Class<? extends Annotation> annotationType : getAnnotationTypes()) {
+
+                    AnnotationAttributes attributes = getAnnotationAttributes(bridgedMethod, annotationType, getEnvironment(), true, true);
+
+                    if (attributes != null && method.equals(ClassUtils.getMostSpecificMethod(method, beanClass))) {
+                        if (Modifier.isStatic(method.getModifiers())) {
+                            if (logger.isWarnEnabled()) {
+                                logger.warn("@" + annotationType.getName() + " annotation is not supported on static methods: " + method);
+                            }
+                            return;
+                        }
+                        if (method.getParameterTypes().length == 0) {
+                            if (logger.isWarnEnabled()) {
+                                logger.warn("@" + annotationType.getName() + " annotation should only be used on methods with parameters: " +
+                                        method);
+                            }
+                        }
+                        PropertyDescriptor pd = BeanUtils.findPropertyForMethod(bridgedMethod, beanClass);
+                        elements.add(new AnnotatedMethodElement(method, pd, attributes));
+                    }
+                }
+            }
+        });
+
+        return elements;
+    }
+
+
+    private AbstractAnnotationBeanPostProcessor.AnnotatedInjectionMetadata buildAnnotatedMetadata(final Class<?> beanClass) {
+        Collection<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> fieldElements = findFieldAnnotationMetadata(beanClass);
+        Collection<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> methodElements = findAnnotatedMethodMetadata(beanClass);
+        return new AbstractAnnotationBeanPostProcessor.AnnotatedInjectionMetadata(beanClass, fieldElements, methodElements);
+    }
+
+    protected AnnotatedInjectionMetadata findInjectionMetadata(String beanName, Class<?> clazz, PropertyValues pvs) {
+        // Fall back to class name as cache key, for backwards compatibility with custom callers.
+        String cacheKey = (StringUtils.hasLength(beanName) ? beanName : clazz.getName());
+        // Quick check on the concurrent map first, with minimal locking.
+        AbstractAnnotationBeanPostProcessor.AnnotatedInjectionMetadata metadata = this.injectionMetadataCache.get(cacheKey);
+        if (InjectionMetadata.needsRefresh(metadata, clazz)) {
+            synchronized (this.injectionMetadataCache) {
+                metadata = this.injectionMetadataCache.get(cacheKey);
+                if (InjectionMetadata.needsRefresh(metadata, clazz)) {
+                    if (metadata != null) {
+                        metadata.clear(pvs);
+                    }
+                    try {
+                        metadata = buildAnnotatedMetadata(clazz);
+                        this.injectionMetadataCache.put(cacheKey, metadata);
+                    } catch (NoClassDefFoundError err) {
+                        throw new IllegalStateException("Failed to introspect object class [" + clazz.getName() +
+                                "] for annotation metadata: could not find class that it depends on", err);
+                    }
+                }
+            }
+        }
+        return metadata;
+    }
+
+    @Override
+    public int getOrder() {
+        return order;
+    }
+
+    public void setOrder(int order) {
+        this.order = order;
+    }
+
+    @Override
+    public void destroy() throws Exception {
+
+        for (Object object : injectedObjectsCache.values()) {
+            if (logger.isInfoEnabled()) {
+                logger.info(object + " was destroying!");
+            }
+
+            if (object instanceof DisposableBean) {
+                ((DisposableBean) object).destroy();
+            }
+        }
+
+        injectionMetadataCache.clear();
+        injectedObjectsCache.clear();
+
+        if (logger.isInfoEnabled()) {
+            logger.info(getClass() + " was destroying!");
+        }
+
+    }
+
+    @Override
+    public void setBeanClassLoader(ClassLoader classLoader) {
+        this.classLoader = classLoader;
+    }
+
+    @Override
+    public void setEnvironment(Environment environment) {
+        this.environment = environment;
+    }
+
+    protected Environment getEnvironment() {
+        return environment;
+    }
+
+    protected ClassLoader getClassLoader() {
+        return classLoader;
+    }
+
+    protected ConfigurableListableBeanFactory getBeanFactory() {
+        return beanFactory;
+    }
+
+    /**
+     * Get injected-object from specified {@link AnnotationAttributes annotation attributes} and Bean Class
+     *
+     * @param attributes      {@link AnnotationAttributes the annotation attributes}
+     * @param bean            Current bean that will be injected
+     * @param beanName        Current bean name that will be injected
+     * @param injectedType    the type of injected-object
+     * @param injectedElement {@link AnnotatedInjectElement}
+     * @return An injected object
+     * @throws Exception If getting is failed
+     */
+    protected Object getInjectedObject(AnnotationAttributes attributes, Object bean, String beanName, Class<?> injectedType,
+                                       AnnotatedInjectElement injectedElement) throws Exception {
+
+        String cacheKey = buildInjectedObjectCacheKey(attributes, bean, beanName, injectedType, injectedElement);
+
+        Object injectedObject = injectedObjectsCache.get(cacheKey);
+
+        if (injectedObject == null) {
+            injectedObject = doGetInjectedBean(attributes, bean, beanName, injectedType, injectedElement);
+            // Customized inject-object if necessary
+            injectedObjectsCache.put(cacheKey, injectedObject);
+        }
+
+        return injectedObject;
+
+    }
+
+    /**
+     * Prepare injection data after found injection elements
+     * @param metadata
+     * @throws Exception
+     */
+    protected void prepareInjection(AnnotatedInjectionMetadata metadata) throws Exception {
+    }
+
+    /**
+     * Subclass must implement this method to get injected-object. The context objects could help this method if
+     * necessary :
+     * <ul>
+     * <li>{@link #getBeanFactory() BeanFactory}</li>
+     * <li>{@link #getClassLoader() ClassLoader}</li>
+     * <li>{@link #getEnvironment() Environment}</li>
+     * </ul>
+     *
+     * @param attributes      {@link AnnotationAttributes the annotation attributes}
+     * @param bean            Current bean that will be injected
+     * @param beanName        Current bean name that will be injected
+     * @param injectedType    the type of injected-object
+     * @param injectedElement {@link AnnotatedInjectElement}
+     * @return The injected object
+     * @throws Exception If resolving an injected object is failed.
+     */
+    protected abstract Object doGetInjectedBean(AnnotationAttributes attributes, Object bean, String beanName, Class<?> injectedType,
+                                                AnnotatedInjectElement injectedElement) throws Exception;
+
+    /**
+     * Build a cache key for injected-object. The context objects could help this method if
+     * necessary :
+     * <ul>
+     * <li>{@link #getBeanFactory() BeanFactory}</li>
+     * <li>{@link #getClassLoader() ClassLoader}</li>
+     * <li>{@link #getEnvironment() Environment}</li>
+     * </ul>
+     *
+     * @param attributes      {@link AnnotationAttributes the annotation attributes}
+     * @param bean            Current bean that will be injected
+     * @param beanName        Current bean name that will be injected
+     * @param injectedType    the type of injected-object
+     * @param injectedElement {@link AnnotatedInjectElement}
+     * @return Bean cache key
+     */
+    protected abstract String buildInjectedObjectCacheKey(AnnotationAttributes attributes, Object bean, String beanName,
+                                                          Class<?> injectedType,
+                                                          AnnotatedInjectElement injectedElement);
+
+    /**
+     * {@link Annotation Annotated} {@link InjectionMetadata} implementation
+     */
+    protected class AnnotatedInjectionMetadata extends InjectionMetadata {
+
+        private final Collection<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> fieldElements;
+
+        private final Collection<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> methodElements;
+
+        public AnnotatedInjectionMetadata(Class<?> targetClass, Collection<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> fieldElements,
+                                          Collection<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> methodElements) {
+            super(targetClass, combine(fieldElements, methodElements));
+            this.fieldElements = fieldElements;
+            this.methodElements = methodElements;
+        }
+
+        public Collection<AbstractAnnotationBeanPostProcessor.AnnotatedFieldElement> getFieldElements() {
+            return fieldElements;
+        }
+
+        public Collection<AbstractAnnotationBeanPostProcessor.AnnotatedMethodElement> getMethodElements() {
+            return methodElements;
+        }
+    }
+
+    /**
+     * {@link Annotation Annotated} {@link Method} {@link InjectionMetadata.InjectedElement}
+     */
+    protected class AnnotatedInjectElement extends InjectionMetadata.InjectedElement {
+
+        protected final AnnotationAttributes attributes;
+
+        protected volatile String refKey;
+
+        protected AnnotatedInjectElement(Member member, PropertyDescriptor pd, AnnotationAttributes attributes) {
+            super(member, pd);
+            this.attributes = attributes;
+        }
+
+        @Override
+        protected void inject(Object bean, String beanName, PropertyValues pvs) throws Throwable {
+
+            Class<?> injectedType = getResourceType();
+            Object injectedObject = getInjectedObject(attributes, bean, beanName, injectedType, this);
+
+            if (member instanceof Field) {
+                Field field = (Field) member;
+                ReflectionUtils.makeAccessible(field);
+                field.set(bean, injectedObject);
+            } else if (member instanceof Method) {
+                Method method = (Method) member;
+                ReflectionUtils.makeAccessible(method);
+                method.invoke(bean, injectedObject);
+            }
+        }
+
+        public Class<?> getInjectedType() {
+            return getResourceType();
+        }
+    }
+
+    protected class AnnotatedMethodElement extends AnnotatedInjectElement {
+
+        protected final Method method;
+
+        protected AnnotatedMethodElement(Method method, PropertyDescriptor pd, AnnotationAttributes attributes) {
+            super(method, pd, attributes);
+            this.method = method;
+        }
+    }
+
+    public class AnnotatedFieldElement extends AnnotatedInjectElement {
+
+        protected final Field field;
+
+        protected AnnotatedFieldElement(Field field, AnnotationAttributes attributes) {
+            super(field, null, attributes);
+            this.field = field;
+        }
+
+    }
+}
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationConfigBeanBuilder.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationConfigBeanBuilder.java
deleted file mode 100644
index 01186af..0000000
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AbstractAnnotationConfigBeanBuilder.java
+++ /dev/null
@@ -1,215 +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.config.spring.beans.factory.annotation;
-
-import org.apache.dubbo.config.AbstractInterfaceConfig;
-import org.apache.dubbo.config.ApplicationConfig;
-import org.apache.dubbo.config.ModuleConfig;
-import org.apache.dubbo.config.MonitorConfig;
-import org.apache.dubbo.config.RegistryConfig;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.springframework.context.ApplicationContext;
-import org.springframework.util.Assert;
-
-import java.lang.annotation.Annotation;
-import java.util.List;
-
-import static com.alibaba.spring.util.BeanFactoryUtils.getBeans;
-import static com.alibaba.spring.util.BeanFactoryUtils.getOptionalBean;
-
-/**
- * Abstract Configurable {@link Annotation} Bean Builder
- *
- * @since 2.5.7
- * @deprecated use {@link AnnotatedInterfaceConfigBeanBuilder}
- */
-@Deprecated
-abstract class AbstractAnnotationConfigBeanBuilder<A extends Annotation, B extends AbstractInterfaceConfig> {
-
-    protected final Log logger = LogFactory.getLog(getClass());
-
-    protected final A annotation;
-
-    protected final ApplicationContext applicationContext;
-
-    protected final ClassLoader classLoader;
-
-    protected Object bean;
-
-    protected Class<?> interfaceClass;
-
-    protected AbstractAnnotationConfigBeanBuilder(A annotation, ClassLoader classLoader,
-                                                  ApplicationContext applicationContext) {
-        Assert.notNull(annotation, "The Annotation must not be null!");
-        Assert.notNull(classLoader, "The ClassLoader must not be null!");
-        Assert.notNull(applicationContext, "The ApplicationContext must not be null!");
-        this.annotation = annotation;
-        this.applicationContext = applicationContext;
-        this.classLoader = classLoader;
-
-    }
-
-    /**
-     * Build {@link B}
-     *
-     * @return non-null
-     * @throws Exception
-     */
-    public final B build() throws Exception {
-
-        checkDependencies();
-
-        B bean = doBuild();
-
-        configureBean(bean);
-
-        if (logger.isInfoEnabled()) {
-            logger.info("The bean[type:" + bean.getClass().getSimpleName() + "] has been built.");
-        }
-
-        return bean;
-
-    }
-
-    private void checkDependencies() {
-
-    }
-
-    /**
-     * Builds {@link B Bean}
-     *
-     * @return {@link B Bean}
-     */
-    protected abstract B doBuild();
-
-
-    protected void configureBean(B bean) throws Exception {
-
-        preConfigureBean(annotation, bean);
-
-        configureRegistryConfigs(bean);
-
-        configureMonitorConfig(bean);
-
-        configureApplicationConfig(bean);
-
-        configureModuleConfig(bean);
-
-        postConfigureBean(annotation, bean);
-
-    }
-
-    protected abstract void preConfigureBean(A annotation, B bean) throws Exception;
-
-
-    private void configureRegistryConfigs(B bean) {
-
-        String[] registryConfigBeanIds = resolveRegistryConfigBeanNames(annotation);
-
-        List<RegistryConfig> registryConfigs = getBeans(applicationContext, registryConfigBeanIds, RegistryConfig.class);
-
-        bean.setRegistries(registryConfigs);
-
-    }
-
-    private void configureMonitorConfig(B bean) {
-
-        String monitorBeanName = resolveMonitorConfigBeanName(annotation);
-
-        MonitorConfig monitorConfig = getOptionalBean(applicationContext, monitorBeanName, MonitorConfig.class);
-
-        bean.setMonitor(monitorConfig);
-
-    }
-
-    private void configureApplicationConfig(B bean) {
-
-        String applicationConfigBeanName = resolveApplicationConfigBeanName(annotation);
-
-        ApplicationConfig applicationConfig =
-                getOptionalBean(applicationContext, applicationConfigBeanName, ApplicationConfig.class);
-
-        bean.setApplication(applicationConfig);
-
-    }
-
-    private void configureModuleConfig(B bean) {
-
-        String moduleConfigBeanName = resolveModuleConfigBeanName(annotation);
-
-        ModuleConfig moduleConfig =
-                getOptionalBean(applicationContext, moduleConfigBeanName, ModuleConfig.class);
-
-        bean.setModule(moduleConfig);
-
-    }
-
-    /**
-     * Resolves the bean name of {@link ModuleConfig}
-     *
-     * @param annotation {@link A}
-     * @return
-     */
-    protected abstract String resolveModuleConfigBeanName(A annotation);
-
-    /**
-     * Resolves the bean name of {@link ApplicationConfig}
-     *
-     * @param annotation {@link A}
-     * @return
-     */
-    protected abstract String resolveApplicationConfigBeanName(A annotation);
-
-
-    /**
-     * Resolves the bean ids of {@link RegistryConfig}
-     *
-     * @param annotation {@link A}
-     * @return non-empty array
-     */
-    protected abstract String[] resolveRegistryConfigBeanNames(A annotation);
-
-    /**
-     * Resolves the bean name of {@link MonitorConfig}
-     *
-     * @param annotation {@link A}
-     * @return
-     */
-    protected abstract String resolveMonitorConfigBeanName(A annotation);
-
-    /**
-     * Configures Bean
-     *
-     * @param annotation
-     * @param bean
-     */
-    protected abstract void postConfigureBean(A annotation, B bean) throws Exception;
-
-
-    public <T extends AbstractAnnotationConfigBeanBuilder<A, B>> T bean(Object bean) {
-        this.bean = bean;
-        return (T) this;
-    }
-
-    public <T extends AbstractAnnotationConfigBeanBuilder<A, B>> T interfaceClass(Class<?> interfaceClass) {
-        this.interfaceClass = interfaceClass;
-        return (T) this;
-    }
-
-}
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotatedInterfaceConfigBeanBuilder.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotatedInterfaceConfigBeanBuilder.java
deleted file mode 100644
index be951ae..0000000
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotatedInterfaceConfigBeanBuilder.java
+++ /dev/null
@@ -1,215 +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.config.spring.beans.factory.annotation;
-
-import org.apache.dubbo.config.AbstractInterfaceConfig;
-import org.apache.dubbo.config.ApplicationConfig;
-import org.apache.dubbo.config.ModuleConfig;
-import org.apache.dubbo.config.MonitorConfig;
-import org.apache.dubbo.config.RegistryConfig;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.springframework.context.ApplicationContext;
-import org.springframework.core.annotation.AnnotationAttributes;
-import org.springframework.util.Assert;
-
-import java.lang.annotation.Annotation;
-import java.util.List;
-
-import static com.alibaba.spring.util.BeanFactoryUtils.getBeans;
-import static com.alibaba.spring.util.BeanFactoryUtils.getOptionalBean;
-
-/**
- * An Abstract Builder to build {@link AbstractInterfaceConfig Interface Config} Bean that annotated
- * some {@link Annotation annotation}.
- *
- * @see ReferenceBeanBuilder
- * @see AbstractInterfaceConfig
- * @see AnnotationAttributes
- * @since 2.7.3
- */
-public abstract class AnnotatedInterfaceConfigBeanBuilder<C extends AbstractInterfaceConfig> {
-
-    protected final Log logger = LogFactory.getLog(getClass());
-
-    protected final AnnotationAttributes attributes;
-
-    protected final ApplicationContext applicationContext;
-
-    protected final ClassLoader classLoader;
-
-    protected Object configBean;
-
-    protected Class<?> interfaceClass;
-
-    protected AnnotatedInterfaceConfigBeanBuilder(AnnotationAttributes attributes, ApplicationContext applicationContext) {
-        Assert.notNull(attributes, "The Annotation attributes must not be null!");
-        Assert.notNull(applicationContext, "The ApplicationContext must not be null!");
-        this.attributes = attributes;
-        this.applicationContext = applicationContext;
-        this.classLoader = applicationContext.getClassLoader() != null ?
-                applicationContext.getClassLoader() : Thread.currentThread().getContextClassLoader();
-    }
-
-    /**
-     * Build {@link C}
-     *
-     * @return non-null
-     * @throws Exception
-     */
-    public final C build() throws Exception {
-
-        checkDependencies();
-
-        C configBean = doBuild();
-
-        configureBean(configBean);
-
-        if (logger.isInfoEnabled()) {
-            logger.info("The configBean[type:" + configBean.getClass().getSimpleName() + "] has been built.");
-        }
-
-        return configBean;
-
-    }
-
-    private void checkDependencies() {
-
-    }
-
-    /**
-     * Builds {@link C Bean}
-     *
-     * @return {@link C Bean}
-     */
-    protected abstract C doBuild();
-
-
-    protected void configureBean(C configBean) throws Exception {
-
-        preConfigureBean(attributes, configBean);
-
-        configureRegistryConfigs(configBean);
-
-        configureMonitorConfig(configBean);
-
-        configureApplicationConfig(configBean);
-
-        configureModuleConfig(configBean);
-
-        postConfigureBean(attributes, configBean);
-
-    }
-
-    protected abstract void preConfigureBean(AnnotationAttributes attributes, C configBean) throws Exception;
-
-
-    private void configureRegistryConfigs(C configBean) {
-
-        String[] registryConfigBeanIds = resolveRegistryConfigBeanNames(attributes);
-
-        List<RegistryConfig> registryConfigs = getBeans(applicationContext, registryConfigBeanIds, RegistryConfig.class);
-
-        configBean.setRegistries(registryConfigs);
-
-    }
-
-    private void configureMonitorConfig(C configBean) {
-
-        String monitorBeanName = resolveMonitorConfigBeanName(attributes);
-
-        MonitorConfig monitorConfig = getOptionalBean(applicationContext, monitorBeanName, MonitorConfig.class);
-
-        configBean.setMonitor(monitorConfig);
-
-    }
-
-    private void configureApplicationConfig(C configBean) {
-
-        String applicationConfigBeanName = resolveApplicationConfigBeanName(attributes);
-
-        ApplicationConfig applicationConfig =
-                getOptionalBean(applicationContext, applicationConfigBeanName, ApplicationConfig.class);
-
-        configBean.setApplication(applicationConfig);
-
-    }
-
-    private void configureModuleConfig(C configBean) {
-
-        String moduleConfigBeanName = resolveModuleConfigBeanName(attributes);
-
-        ModuleConfig moduleConfig =
-                getOptionalBean(applicationContext, moduleConfigBeanName, ModuleConfig.class);
-
-        configBean.setModule(moduleConfig);
-
-    }
-
-    /**
-     * Resolves the configBean name of {@link ModuleConfig}
-     *
-     * @param attributes {@link AnnotationAttributes}
-     * @return
-     */
-    protected abstract String resolveModuleConfigBeanName(AnnotationAttributes attributes);
-
-    /**
-     * Resolves the configBean name of {@link ApplicationConfig}
-     *
-     * @param attributes {@link AnnotationAttributes}
-     * @return
-     */
-    protected abstract String resolveApplicationConfigBeanName(AnnotationAttributes attributes);
-
-
-    /**
-     * Resolves the configBean ids of {@link RegistryConfig}
-     *
-     * @param attributes {@link AnnotationAttributes}
-     * @return non-empty array
-     */
-    protected abstract String[] resolveRegistryConfigBeanNames(AnnotationAttributes attributes);
-
-    /**
-     * Resolves the configBean name of {@link MonitorConfig}
-     *
-     * @param attributes {@link AnnotationAttributes}
-     * @return
-     */
-    protected abstract String resolveMonitorConfigBeanName(AnnotationAttributes attributes);
-
-    /**
-     * Configures Bean
-     *
-     * @param attributes
-     * @param configBean
-     */
-    protected abstract void postConfigureBean(AnnotationAttributes attributes, C configBean) throws Exception;
-
-
-    public <T extends AnnotatedInterfaceConfigBeanBuilder<C>> T configBean(Object configBean) {
-        this.configBean = configBean;
-        return (T) this;
-    }
-
-    public <T extends AnnotatedInterfaceConfigBeanBuilder<C>> T interfaceClass(Class<?> interfaceClass) {
-        this.interfaceClass = interfaceClass;
-        return (T) this;
-    }
-}
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapter.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapter.java
index 80cdfb7..88280a5 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapter.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapter.java
@@ -33,7 +33,7 @@ import static com.alibaba.spring.util.AnnotationUtils.getAttributes;
  * @see PropertyValues
  * @since 2.5.11
  */
-class AnnotationPropertyValuesAdapter implements PropertyValues {
+public class AnnotationPropertyValuesAdapter implements PropertyValues {
 
     private final PropertyValues delegate;
 
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessor.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessor.java
index 07de2ee..7255264 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessor.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessor.java
@@ -16,33 +16,44 @@
  */
 package org.apache.dubbo.config.spring.beans.factory.annotation;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.dubbo.common.utils.ClassUtils;
 import org.apache.dubbo.config.annotation.DubboReference;
-import org.apache.dubbo.config.annotation.DubboService;
 import org.apache.dubbo.config.annotation.Reference;
 import org.apache.dubbo.config.annotation.Service;
+import org.apache.dubbo.config.context.ConfigManager;
 import org.apache.dubbo.config.spring.ReferenceBean;
+import org.apache.dubbo.config.spring.ReferenceBeanManager;
 import org.apache.dubbo.config.spring.ServiceBean;
-
-import com.alibaba.spring.beans.factory.annotation.AbstractAnnotationBeanPostProcessor;
+import org.apache.dubbo.config.spring.util.DubboBeanUtils;
 import org.springframework.beans.BeansException;
+import org.springframework.beans.PropertyValue;
+import org.springframework.beans.PropertyValues;
+import org.springframework.beans.factory.BeanCreationException;
 import org.springframework.beans.factory.annotation.InjectionMetadata;
+import org.springframework.beans.factory.config.BeanDefinition;
 import org.springframework.beans.factory.config.ConfigurableListableBeanFactory;
-import org.springframework.beans.factory.config.RuntimeBeanReference;
-import org.springframework.beans.factory.support.AbstractBeanDefinition;
+import org.springframework.beans.factory.support.BeanDefinitionRegistry;
+import org.springframework.beans.factory.support.BeanDefinitionRegistryPostProcessor;
+import org.springframework.beans.factory.support.RootBeanDefinition;
 import org.springframework.context.ApplicationContext;
 import org.springframework.context.ApplicationContextAware;
 import org.springframework.core.annotation.AnnotationAttributes;
+import org.springframework.util.Assert;
+import org.springframework.util.ObjectUtils;
 
-import java.lang.reflect.Field;
-import java.lang.reflect.Method;
+import java.beans.PropertyDescriptor;
+import java.util.ArrayList;
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
 
 import static com.alibaba.spring.util.AnnotationUtils.getAttribute;
-import static com.alibaba.spring.util.AnnotationUtils.getAttributes;
 import static org.apache.dubbo.config.spring.beans.factory.annotation.ServiceBeanNameBuilder.create;
 import static org.springframework.util.StringUtils.hasText;
 
@@ -55,8 +66,8 @@ import static org.springframework.util.StringUtils.hasText;
  * @see com.alibaba.dubbo.config.annotation.Reference
  * @since 2.5.7
  */
-public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBeanPostProcessor implements
-        ApplicationContextAware {
+public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBeanPostProcessor
+        implements ApplicationContextAware, BeanDefinitionRegistryPostProcessor {
 
     /**
      * The bean name of {@link ReferenceAnnotationBeanPostProcessor}
@@ -68,8 +79,7 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
      */
     private static final int CACHE_SIZE = Integer.getInteger(BEAN_NAME + ".cache.size", 32);
 
-    private final ConcurrentMap<String, ReferenceBean<?>> referenceBeanCache =
-            new ConcurrentHashMap<>(CACHE_SIZE);
+    private final Log logger = LogFactory.getLog(getClass());
 
     private final ConcurrentMap<InjectionMetadata.InjectedElement, ReferenceBean<?>> injectedFieldReferenceBeanCache =
             new ConcurrentHashMap<>(CACHE_SIZE);
@@ -79,6 +89,9 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
 
     private ApplicationContext applicationContext;
 
+    private ReferenceBeanManager referenceBeanManager;
+    private BeanDefinitionRegistry beanDefinitionRegistry;
+
     /**
      * {@link com.alibaba.dubbo.config.annotation.Reference @com.alibaba.dubbo.config.annotation.Reference} has been supported since 2.7.3
      * <p>
@@ -88,96 +101,175 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
         super(DubboReference.class, Reference.class, com.alibaba.dubbo.config.annotation.Reference.class);
     }
 
-    /**
-     * Gets all beans of {@link ReferenceBean}
-     *
-     * @return non-null read-only {@link Collection}
-     * @since 2.5.9
-     */
-    public Collection<ReferenceBean<?>> getReferenceBeans() {
-        return referenceBeanCache.values();
+    @Override
+    public void postProcessBeanDefinitionRegistry(BeanDefinitionRegistry registry) throws BeansException {
+        this.beanDefinitionRegistry = registry;
     }
 
-    /**
-     * Get {@link ReferenceBean} {@link Map} in injected field.
-     *
-     * @return non-null {@link Map}
-     * @since 2.5.11
-     */
-    public Map<InjectionMetadata.InjectedElement, ReferenceBean<?>> getInjectedFieldReferenceBeanMap() {
-        return Collections.unmodifiableMap(injectedFieldReferenceBeanCache);
+    @Override
+    public void postProcessBeanFactory(ConfigurableListableBeanFactory beanFactory) throws BeansException {
+        DubboBeanUtils.registerBeansIfNotExists(beanDefinitionRegistry);
+
+        String[] beanNames = beanFactory.getBeanDefinitionNames();
+        for (String beanName : beanNames) {
+            Class<?> beanType;
+            if (beanFactory.isFactoryBean(beanName)){
+                BeanDefinition beanDefinition = beanFactory.getMergedBeanDefinition(beanName);
+                if (isReferenceBean(beanDefinition)) {
+                    continue;
+                }
+                String beanClassName = beanDefinition.getBeanClassName();
+                beanType = ClassUtils.resolveClass(beanClassName, getClassLoader());
+            } else {
+                beanType = beanFactory.getType(beanName);
+            }
+            if (beanType != null) {
+                AnnotatedInjectionMetadata metadata = findInjectionMetadata(beanName, beanType, null);
+                try {
+                    prepareInjection(metadata);
+                } catch (Exception e) {
+                    logger.warn("Prepare dubbo reference injection element failed", e);
+                }
+            }
+        }
     }
 
-    /**
-     * Get {@link ReferenceBean} {@link Map} in injected method.
-     *
-     * @return non-null {@link Map}
-     * @since 2.5.11
-     */
-    public Map<InjectionMetadata.InjectedElement, ReferenceBean<?>> getInjectedMethodReferenceBeanMap() {
-        return Collections.unmodifiableMap(injectedMethodReferenceBeanCache);
+    @Override
+    public void postProcessMergedBeanDefinition(RootBeanDefinition beanDefinition, Class<?> beanType, String beanName) {
+        if (beanType != null) {
+            if (isReferenceBean(beanDefinition)) {
+                //mark property value as optional
+                List<PropertyValue> propertyValues = beanDefinition.getPropertyValues().getPropertyValueList();
+                for (PropertyValue propertyValue : propertyValues) {
+                    propertyValue.setOptional(true);
+                }
+            } else {
+                AnnotatedInjectionMetadata metadata = findInjectionMetadata(beanName, beanType, null);
+                metadata.checkConfigMembers(beanDefinition);
+                try {
+                    prepareInjection(metadata);
+                } catch (Exception e) {
+                    logger.warn("Prepare dubbo reference injection element failed", e);
+                }
+            }
+        }
     }
 
     @Override
-    protected Object doGetInjectedBean(AnnotationAttributes attributes, Object bean, String beanName, Class<?> injectedType,
-                                       InjectionMetadata.InjectedElement injectedElement) throws Exception {
-        /**
-         * The name of bean that annotated Dubbo's {@link Service @Service} in local Spring {@link ApplicationContext}
-         */
-        String referencedBeanName = buildReferencedBeanName(attributes, injectedType);
-
-        /**
-         * The name of bean that is declared by {@link Reference @Reference} annotation injection
-         */
-        String referenceBeanName = getReferenceBeanName(attributes, injectedType);
+    public PropertyValues postProcessPropertyValues(
+            PropertyValues pvs, PropertyDescriptor[] pds, Object bean, String beanName) throws BeanCreationException {
+
+        try {
+            AnnotatedInjectionMetadata metadata = findInjectionMetadata(beanName, bean.getClass(), pvs);
+            prepareInjection(metadata);
+            metadata.inject(bean, beanName, pvs);
+        } catch (BeanCreationException ex) {
+            throw ex;
+        } catch (Throwable ex) {
+            throw new BeanCreationException(beanName, "Injection of @" + getAnnotationType().getSimpleName()
+                    + " dependencies is failed", ex);
+        }
+        return pvs;
+    }
 
-        ReferenceBean referenceBean = buildReferenceBeanIfAbsent(referenceBeanName, attributes, injectedType);
+    private boolean isReferenceBean(BeanDefinition beanDefinition) {
+        return ReferenceBean.class.getName().equals(beanDefinition.getBeanClassName());
+    }
 
-        boolean localServiceBean = isLocalServiceBean(referencedBeanName, referenceBean, attributes);
+    protected void prepareInjection(AnnotatedInjectionMetadata metadata) throws Exception {
+        //find and registry bean definition for @DubboReference/@Reference
+        for (AnnotatedFieldElement fieldElement : metadata.getFieldElements()) {
+            if (fieldElement.refKey != null) {
+                continue;
+            }
+            Class<?> injectedType = fieldElement.field.getType();
+            AnnotationAttributes attributes = fieldElement.attributes;
+            ReferenceBean referenceBean = getReferenceBean(injectedType, attributes);
 
-        prepareReferenceBean(referencedBeanName, referenceBean, localServiceBean);
+            //associate fieldElement and reference bean
+            fieldElement.refKey = referenceBean.getId();
+            injectedFieldReferenceBeanCache.put(fieldElement, referenceBean);
 
-        registerReferenceBean(referencedBeanName, referenceBean, attributes, localServiceBean, injectedType);
+        }
 
-        cacheInjectedReferenceBean(referenceBean, injectedElement);
+        for (AnnotatedMethodElement methodElement : metadata.getMethodElements()) {
+            if (methodElement.refKey != null) {
+                continue;
+            }
+            Class<?> injectedType = methodElement.getInjectedType();
+            AnnotationAttributes attributes = methodElement.attributes;
+            ReferenceBean referenceBean = getReferenceBean(injectedType, attributes);
 
-        return referenceBean.get();
+            //associate fieldElement and reference bean
+            methodElement.refKey = referenceBean.getId();
+            injectedMethodReferenceBeanCache.put(methodElement, referenceBean);
+        }
     }
 
-    /**
-     * Register an instance of {@link ReferenceBean} as a Spring Bean
-     *
-     * @param referencedBeanName The name of bean that annotated Dubbo's {@link Service @Service} in the Spring {@link ApplicationContext}
-     * @param referenceBean      the instance of {@link ReferenceBean} is about to register into the Spring {@link ApplicationContext}
-     * @param attributes         the {@link AnnotationAttributes attributes} of {@link Reference @Reference}
-     * @param localServiceBean   Is Local Service bean or not
-     * @param interfaceClass     the {@link Class class} of Service interface
-     * @since 2.7.3
-     */
-    private void registerReferenceBean(String referencedBeanName, ReferenceBean referenceBean,
-                                       AnnotationAttributes attributes,
-                                       boolean localServiceBean, Class<?> interfaceClass) {
-
-        ConfigurableListableBeanFactory beanFactory = getBeanFactory();
+    private ReferenceBean getReferenceBean(Class<?> injectedType, AnnotationAttributes attributes) throws Exception {
+        // referenceBeanName
+        String referenceBeanName = getReferenceBeanName(attributes, injectedType);
 
-        String beanName = getReferenceBeanName(attributes, interfaceClass);
+        // reuse exist reference bean?
+        ReferenceBean referenceBean = referenceBeanManager.get(referenceBeanName);
 
-        if (localServiceBean) {  // If @Service bean is local one
+        //create referenceBean
+        if (referenceBean == null) {
+            //handle injvm/localServiceBean
             /**
-             * Get  the @Service's BeanDefinition from {@link BeanFactory}
-             * Refer to {@link ServiceAnnotationBeanPostProcessor#buildServiceBeanDefinition}
+             * The name of bean that annotated Dubbo's {@link Service @Service} in local Spring {@link ApplicationContext}
              */
-            AbstractBeanDefinition beanDefinition = (AbstractBeanDefinition) beanFactory.getBeanDefinition(referencedBeanName);
-            RuntimeBeanReference runtimeBeanReference = (RuntimeBeanReference) beanDefinition.getPropertyValues().get("ref");
-            // The name of bean annotated @Service
-            String serviceBeanName = runtimeBeanReference.getBeanName();
-            // register Alias rather than a new bean name, in order to reduce duplicated beans
-            beanFactory.registerAlias(serviceBeanName, beanName);
-        } else { // Remote @Service Bean
-            if (!beanFactory.containsBean(beanName)) {
-                beanFactory.registerSingleton(beanName, referenceBean);
+            String localServiceBeanName = buildReferencedBeanName(attributes, injectedType);
+            boolean localServiceBean = isLocalServiceBean(localServiceBeanName, attributes);
+            if (localServiceBean) { // If the local @Service Bean exists
+                attributes.put("injvm", Boolean.TRUE);
+                //  Issue : https://github.com/apache/dubbo/issues/6224
+                //exportServiceBeanIfNecessary(localServiceBeanName); // If the referenced ServiceBean exits, export it immediately
+            }
+
+            //check interfaceClass
+            if (attributes.get("interfaceName") == null && attributes.get("interfaceClass") == null) {
+                Class<?> interfaceClass = injectedType;
+                Assert.isTrue(interfaceClass.isInterface(),
+                        "The class of field or method that was annotated @DubboReference is not an interface!");
+                attributes.put("interfaceClass", interfaceClass);
+            }
+
+            //init reference bean
+            try {
+                //registry referenceBean
+                RootBeanDefinition beanDefinition = new RootBeanDefinition();
+                beanDefinition.setBeanClassName(ReferenceBean.class.getName());
+                //set autowireCandidate to false for local call, avoiding multiple candidate beans for @Autowire
+                beanDefinition.setAutowireCandidate(!localServiceBean);
+                //beanDefinition.getPropertyValues()
+
+                referenceBean = new ReferenceBean(attributes);
+                referenceBean.setId(referenceBeanName);
+                referenceBean.setApplicationContext(applicationContext);
+                referenceBean.setBeanClassLoader(getClassLoader());
+                referenceBean.afterPropertiesSet();
+
+                beanDefinitionRegistry.registerBeanDefinition(referenceBeanName, beanDefinition);
+                getBeanFactory().registerSingleton(referenceBeanName, referenceBean);
+
+                referenceBeanManager.addReference(referenceBean);
+            } catch (Exception e) {
+                throw new Exception("Create dubbo reference bean failed", e);
             }
         }
+        return referenceBean;
+    }
+
+    @Override
+    protected Object doGetInjectedBean(AnnotationAttributes attributes, Object bean, String beanName, Class<?> injectedType,
+                                       AnnotatedInjectElement injectedElement) throws Exception {
+
+        if (injectedElement.refKey == null) {
+            throw new IllegalStateException("The AnnotatedInjectElement of @DubboReference should be inited before injection");
+        }
+
+        return getBeanFactory().getBean(injectedElement.refKey);
     }
 
     /**
@@ -211,10 +303,19 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
 
         if (!attributes.isEmpty()) {
             beanNameBuilder.append('(');
-            for (Map.Entry<String, Object> entry : attributes.entrySet()) {
-                beanNameBuilder.append(entry.getKey())
+            //sort attributes keys
+            List<String> sortedAttrKeys = new ArrayList<>(attributes.keySet());
+            Collections.sort(sortedAttrKeys);
+            for (String key : sortedAttrKeys) {
+                Object value = attributes.get(key);
+                //handle method array, generic array
+                if (value!=null && value.getClass().isArray()) {
+                    Object[] array = ObjectUtils.toObjectArray(value);
+                    value = Arrays.toString(array);
+                }
+                beanNameBuilder.append(key)
                         .append('=')
-                        .append(entry.getValue())
+                        .append(value)
                         .append(',');
             }
             // replace the latest "," to be ")"
@@ -223,6 +324,9 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
 
         beanNameBuilder.append(" ").append(interfaceClass.getName());
 
+        //TODO remove invalid chars
+        //TODO test @DubboReference with Method config
+        //.replaceAll("[<>]", "_")
         return beanNameBuilder.toString();
     }
 
@@ -233,8 +337,8 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
      * @return If the target referenced bean is existed, return <code>true</code>, or <code>false</code>
      * @since 2.7.6
      */
-    private boolean isLocalServiceBean(String referencedBeanName, ReferenceBean referenceBean, AnnotationAttributes attributes) {
-        return existsServiceBean(referencedBeanName) && !isRemoteReferenceBean(referenceBean, attributes);
+    private boolean isLocalServiceBean(String referencedBeanName, AnnotationAttributes attributes) {
+        return existsServiceBean(referencedBeanName) && !isRemoteReferenceBean(attributes);
     }
 
     /**
@@ -250,29 +354,12 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
 
     }
 
-    private boolean isRemoteReferenceBean(ReferenceBean referenceBean, AnnotationAttributes attributes) {
-        boolean remote = Boolean.FALSE.equals(referenceBean.isInjvm()) || Boolean.FALSE.equals(attributes.get("injvm"));
+    private boolean isRemoteReferenceBean(AnnotationAttributes attributes) {
+        //TODO Can the interface be called locally when injvm is empty? https://github.com/apache/dubbo/issues/6842
+        boolean remote = Boolean.FALSE.equals(attributes.get("injvm"));
         return remote;
     }
 
-    /**
-     * Prepare {@link ReferenceBean}
-     *
-     * @param referencedBeanName The name of bean that annotated Dubbo's {@link DubboService @DubboService}
-     *                           in the Spring {@link ApplicationContext}
-     * @param referenceBean      the instance of {@link ReferenceBean}
-     * @param localServiceBean   Is Local Service bean or not
-     * @since 2.7.8
-     */
-    private void prepareReferenceBean(String referencedBeanName, ReferenceBean referenceBean, boolean localServiceBean) {
-        //  Issue : https://github.com/apache/dubbo/issues/6224
-        if (localServiceBean) { // If the local @Service Bean exists
-            referenceBean.setInjvm(Boolean.TRUE);
-            exportServiceBeanIfNecessary(referencedBeanName); // If the referenced ServiceBean exits, export it immediately
-        }
-    }
-
-
     private void exportServiceBeanIfNecessary(String referencedBeanName) {
         if (existsServiceBean(referencedBeanName)) {
             ServiceBean serviceBean = getServiceBean(referencedBeanName);
@@ -288,10 +375,8 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
 
     @Override
     protected String buildInjectedObjectCacheKey(AnnotationAttributes attributes, Object bean, String beanName,
-                                                 Class<?> injectedType, InjectionMetadata.InjectedElement injectedElement) {
-        return buildReferencedBeanName(attributes, injectedType) +
-                "#source=" + (injectedElement.getMember()) +
-                "#attributes=" + getAttributes(attributes, getEnvironment());
+                                                 Class<?> injectedType, AnnotatedInjectElement injectedElement) {
+        return generateReferenceBeanName(attributes, injectedType);
     }
 
     /**
@@ -304,44 +389,45 @@ public class ReferenceAnnotationBeanPostProcessor extends AbstractAnnotationBean
         return serviceBeanNameBuilder.build();
     }
 
-    private ReferenceBean buildReferenceBeanIfAbsent(String referenceBeanName, AnnotationAttributes attributes,
-                                                     Class<?> referencedType)
-            throws Exception {
-
-        ReferenceBean<?> referenceBean = referenceBeanCache.get(referenceBeanName);
-
-        if (referenceBean == null) {
-            ReferenceBeanBuilder beanBuilder = ReferenceBeanBuilder
-                    .create(attributes, applicationContext)
-                    .interfaceClass(referencedType);
-            referenceBean = beanBuilder.build();
-            referenceBeanCache.put(referenceBeanName, referenceBean);
-        } else if (!referencedType.isAssignableFrom(referenceBean.getInterfaceClass())) {
-            throw new IllegalArgumentException("reference bean name " + referenceBeanName + " has been duplicated, but interfaceClass " +
-                    referenceBean.getInterfaceClass().getName() + " cannot be assigned to " + referencedType.getName());
-        }
-        return referenceBean;
-    }
-
-    private void cacheInjectedReferenceBean(ReferenceBean referenceBean,
-                                            InjectionMetadata.InjectedElement injectedElement) {
-        if (injectedElement.getMember() instanceof Field) {
-            injectedFieldReferenceBeanCache.put(injectedElement, referenceBean);
-        } else if (injectedElement.getMember() instanceof Method) {
-            injectedMethodReferenceBeanCache.put(injectedElement, referenceBean);
-        }
-    }
-
     @Override
     public void setApplicationContext(ApplicationContext applicationContext) throws BeansException {
         this.applicationContext = applicationContext;
+        this.referenceBeanManager = applicationContext.getBean(ReferenceBeanManager.BEAN_NAME, ReferenceBeanManager.class);
     }
 
     @Override
     public void destroy() throws Exception {
         super.destroy();
-        this.referenceBeanCache.clear();
         this.injectedFieldReferenceBeanCache.clear();
         this.injectedMethodReferenceBeanCache.clear();
     }
+
+    /**
+     * Gets all beans of {@link ReferenceBean}
+     * @deprecated  use {@link ConfigManager#getReferences()} instead
+     */
+    @Deprecated
+    public Collection<ReferenceBean<?>> getReferenceBeans() {
+        return Collections.emptyList();
+    }
+
+    /**
+     * Get {@link ReferenceBean} {@link Map} in injected field.
+     *
+     * @return non-null {@link Map}
+     * @since 2.5.11
+     */
+    public Map<InjectionMetadata.InjectedElement, ReferenceBean<?>> getInjectedFieldReferenceBeanMap() {
+        return Collections.unmodifiableMap(injectedFieldReferenceBeanCache);
+    }
+
+    /**
+     * Get {@link ReferenceBean} {@link Map} in injected method.
+     *
+     * @return non-null {@link Map}
+     * @since 2.5.11
+     */
+    public Map<InjectionMetadata.InjectedElement, ReferenceBean<?>> getInjectedMethodReferenceBeanMap() {
+        return Collections.unmodifiableMap(injectedMethodReferenceBeanCache);
+    }
 }
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilder.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilder.java
index f6aacf4..f58c93b 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilder.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilder.java
@@ -16,13 +16,18 @@
  */
 package org.apache.dubbo.config.spring.beans.factory.annotation;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
 import org.apache.dubbo.common.utils.CollectionUtils;
+import org.apache.dubbo.config.ApplicationConfig;
 import org.apache.dubbo.config.ConsumerConfig;
 import org.apache.dubbo.config.MethodConfig;
+import org.apache.dubbo.config.ModuleConfig;
+import org.apache.dubbo.config.MonitorConfig;
+import org.apache.dubbo.config.ReferenceConfig;
+import org.apache.dubbo.config.RegistryConfig;
+import org.apache.dubbo.config.annotation.DubboReference;
 import org.apache.dubbo.config.annotation.Method;
-import org.apache.dubbo.config.annotation.Reference;
-import org.apache.dubbo.config.spring.ReferenceBean;
-
 import org.springframework.beans.propertyeditors.StringTrimmerEditor;
 import org.springframework.context.ApplicationContext;
 import org.springframework.core.annotation.AnnotationAttributes;
@@ -31,85 +36,195 @@ import org.springframework.util.StringUtils;
 import org.springframework.validation.DataBinder;
 
 import java.beans.PropertyEditorSupport;
+import java.util.Arrays;
 import java.util.List;
 import java.util.Map;
 
 import static com.alibaba.spring.util.AnnotationUtils.getAttribute;
-import static com.alibaba.spring.util.AnnotationUtils.getAttributes;
+import static com.alibaba.spring.util.BeanFactoryUtils.getBeans;
 import static com.alibaba.spring.util.BeanFactoryUtils.getOptionalBean;
 import static com.alibaba.spring.util.ObjectUtils.of;
-import static org.apache.dubbo.config.spring.util.DubboAnnotationUtils.resolveServiceInterfaceClass;
-import static org.springframework.core.annotation.AnnotationAttributes.fromMap;
 import static org.springframework.util.StringUtils.commaDelimitedListToStringArray;
 
 /**
- * {@link ReferenceBean} Builder
+ * {@link ReferenceConfig} Builder for @{@link DubboReference}
  *
- * @since 2.5.7
+ * @since 3.0
  */
-class ReferenceBeanBuilder extends AnnotatedInterfaceConfigBeanBuilder<ReferenceBean> {
+public class ReferenceBeanBuilder {
 
     // Ignore those fields
-    static final String[] IGNORE_FIELD_NAMES = of("application", "module", "consumer", "monitor", "registry");
+    static final String[] IGNORE_FIELD_NAMES = of("application", "module", "consumer", "monitor", "registry", "interfaceClass");
+
+    protected final Log logger = LogFactory.getLog(getClass());
+
+    protected final AnnotationAttributes attributes;
+
+    protected final ApplicationContext applicationContext;
+
+    protected final ClassLoader classLoader;
+
+    protected Class<?> defaultInterfaceClass;
 
     private ReferenceBeanBuilder(AnnotationAttributes attributes, ApplicationContext applicationContext) {
-        super(attributes, applicationContext);
+        Assert.notNull(attributes, "The Annotation attributes must not be null!");
+        Assert.notNull(applicationContext, "The ApplicationContext must not be null!");
+        this.attributes = attributes;
+        this.applicationContext = applicationContext;
+        this.classLoader = applicationContext.getClassLoader() != null ?
+                applicationContext.getClassLoader() : Thread.currentThread().getContextClassLoader();
     }
 
-    private void configureInterface(AnnotationAttributes attributes, ReferenceBean referenceBean) {
-        Boolean generic = getAttribute(attributes, "generic");
-        if (generic != null && generic) {
-            // it's a generic reference
-            String interfaceClassName = getAttribute(attributes, "interfaceName");
-            Assert.hasText(interfaceClassName,
-                    "@Reference interfaceName() must be present when reference a generic service!");
-            referenceBean.setInterface(interfaceClassName);
-            return;
+    public final ReferenceConfig build() throws Exception {
+
+        ReferenceConfig configBean = new ReferenceConfig();
+
+        configureBean(configBean);
+
+        if (logger.isInfoEnabled()) {
+            logger.info("The configBean[type:" + configBean.getClass().getSimpleName() + "] has been built.");
         }
 
-        Class<?> serviceInterfaceClass = resolveServiceInterfaceClass(attributes, interfaceClass);
+        return configBean;
+
+    }
+
+    protected void configureBean(ReferenceConfig configBean) throws Exception {
+
+        populateBean(attributes, configBean);
+
+        configureRegistryConfigs(configBean);
+
+        configureMonitorConfig(configBean);
+
+        configureApplicationConfig(configBean);
+
+        configureModuleConfig(configBean);
 
-        Assert.isTrue(serviceInterfaceClass.isInterface(),
-                "The class of field or method that was annotated @Reference is not an interface!");
+        //interfaceClass
+        configureInterface(attributes, configBean);
 
-        referenceBean.setInterface(serviceInterfaceClass);
+        configureConsumerConfig(attributes, configBean);
+
+        configureMethodConfig(attributes, configBean);
+
+        //bean.setApplicationContext(applicationContext);
+        //bean.afterPropertiesSet();
 
     }
 
+    private void configureRegistryConfigs(ReferenceConfig configBean) {
+
+        String[] registryConfigBeanIds = getAttribute(attributes, "registry");
+
+        List<RegistryConfig> registryConfigs = getBeans(applicationContext, registryConfigBeanIds, RegistryConfig.class);
+
+        configBean.setRegistries(registryConfigs);
+
+    }
+
+    private void configureMonitorConfig(ReferenceConfig configBean) {
+
+        String monitorBeanName = getAttribute(attributes, "monitor");
 
-    private void configureConsumerConfig(AnnotationAttributes attributes, ReferenceBean<?> referenceBean) {
+        MonitorConfig monitorConfig = getOptionalBean(applicationContext, monitorBeanName, MonitorConfig.class);
 
-        String consumerBeanName = getAttribute(attributes, "consumer");
+        configBean.setMonitor(monitorConfig);
 
-        ConsumerConfig consumerConfig = getOptionalBean(applicationContext, consumerBeanName, ConsumerConfig.class);
+    }
+
+    private void configureApplicationConfig(ReferenceConfig configBean) {
+
+        String applicationConfigBeanName = getAttribute(attributes, "application");
 
-        referenceBean.setConsumer(consumerConfig);
+        ApplicationConfig applicationConfig =
+                getOptionalBean(applicationContext, applicationConfigBeanName, ApplicationConfig.class);
+
+        configBean.setApplication(applicationConfig);
 
     }
 
-    void configureMethodConfig(AnnotationAttributes attributes, ReferenceBean<?> referenceBean) {
-        Method[] methods = (Method[]) attributes.get("methods");
-        List<MethodConfig> methodConfigs = MethodConfig.constructMethodConfig(methods);
-        if (!methodConfigs.isEmpty()) {
-            referenceBean.setMethods(methodConfigs);
+    private void configureModuleConfig(ReferenceConfig configBean) {
+
+        String moduleConfigBeanName = getAttribute(attributes, "module");
+
+        ModuleConfig moduleConfig =
+                getOptionalBean(applicationContext, moduleConfigBeanName, ModuleConfig.class);
+
+        configBean.setModule(moduleConfig);
+
+    }
+
+    private void configureInterface(AnnotationAttributes attributes, ReferenceConfig referenceBean) {
+        if (referenceBean.getInterface() == null) {
+
+            Object genericValue = getAttribute(attributes, "generic");
+            String generic = (genericValue != null) ? genericValue.toString() : null;
+            referenceBean.setGeneric(generic);
+
+            String interfaceClassName = getAttribute(attributes, "interfaceName");
+            if (StringUtils.hasText(interfaceClassName)) {
+                referenceBean.setInterface(interfaceClassName);
+            } else {
+                Class<?> interfaceClass = getAttribute(attributes, "interfaceClass");
+                if (void.class.equals(interfaceClass)) { // default or set void.class for purpose.
+                    interfaceClass = null;
+                }
+                if (interfaceClass != null) {
+                    Assert.isTrue(interfaceClass.isInterface(),
+                            "The interfaceClass of @DubboReference is not an interface: "+interfaceClass.getName());
+                }
+                // Not present 'interfaceClass' attribute, use default injection type of annotated
+                if (interfaceClass == null && defaultInterfaceClass != null) {
+                    interfaceClass = defaultInterfaceClass;
+                    Assert.isTrue(interfaceClass.isInterface(),
+                            "The class of field or method that was annotated @DubboReference is not an interface!");
+                }
+                // Convert to interface class name, InterfaceClass will be determined later
+                referenceBean.setInterface(interfaceClass.getName());
+            }
+        }
+    }
+
+
+    private void configureConsumerConfig(AnnotationAttributes attributes, ReferenceConfig<?> referenceBean) {
+        ConsumerConfig consumerConfig = null;
+        Object consumer = getAttribute(attributes, "consumer");
+        if (consumer != null) {
+            if (consumer instanceof String) {
+                consumerConfig = getOptionalBean(applicationContext, (String) consumer, ConsumerConfig.class);
+            } else if (consumer instanceof ConsumerConfig) {
+                consumerConfig = (ConsumerConfig) consumer;
+            } else {
+                throw new IllegalArgumentException("Unexpected 'consumer' attribute value: "+consumer);
+            }
+            referenceBean.setConsumer(consumerConfig);
         }
     }
 
-    @Override
-    protected ReferenceBean doBuild() {
-        return new ReferenceBean<Object>();
+    void configureMethodConfig(AnnotationAttributes attributes, ReferenceConfig<?> referenceBean) {
+        Object value = attributes.get("methods");
+        if (value instanceof Method[]) {
+            Method[] methods = (Method[]) value;
+            List<MethodConfig> methodConfigs = MethodConfig.constructMethodConfig(methods);
+            if (!methodConfigs.isEmpty()) {
+                referenceBean.setMethods(methodConfigs);
+            }
+        } else if (value instanceof MethodConfig[]) {
+            MethodConfig[] methodConfigs = (MethodConfig[]) value;
+            referenceBean.setMethods(Arrays.asList(methodConfigs));
+        }
     }
 
-    @Override
-    protected void preConfigureBean(AnnotationAttributes attributes, ReferenceBean referenceBean) {
-        Assert.notNull(interfaceClass, "The interface class must set first!");
+    protected void populateBean(AnnotationAttributes attributes, ReferenceConfig referenceBean) {
+        Assert.notNull(defaultInterfaceClass, "The default interface class must set first!");
         DataBinder dataBinder = new DataBinder(referenceBean);
         // Register CustomEditors for special fields
         dataBinder.registerCustomEditor(String.class, "filter", new StringTrimmerEditor(true));
         dataBinder.registerCustomEditor(String.class, "listener", new StringTrimmerEditor(true));
         dataBinder.registerCustomEditor(Map.class, "parameters", new PropertyEditorSupport() {
             @Override
-            public void setAsText(String text) throws java.lang.IllegalArgumentException {
+            public void setAsText(String text) throws IllegalArgumentException {
                 // Trim all whitespace
                 String content = StringUtils.trimAllWhitespace(text);
                 if (!StringUtils.hasText(content)) { // No content , ignore directly
@@ -130,49 +245,13 @@ class ReferenceBeanBuilder extends AnnotatedInterfaceConfigBeanBuilder<Reference
 
     }
 
-
-    @Override
-    protected String resolveModuleConfigBeanName(AnnotationAttributes attributes) {
-        return getAttribute(attributes, "module");
-    }
-
-    @Override
-    protected String resolveApplicationConfigBeanName(AnnotationAttributes attributes) {
-        return getAttribute(attributes, "application");
-    }
-
-    @Override
-    protected String[] resolveRegistryConfigBeanNames(AnnotationAttributes attributes) {
-        return getAttribute(attributes, "registry");
-    }
-
-    @Override
-    protected String resolveMonitorConfigBeanName(AnnotationAttributes attributes) {
-        return getAttribute(attributes, "monitor");
-    }
-
-    @Override
-    protected void postConfigureBean(AnnotationAttributes attributes, ReferenceBean bean) throws Exception {
-
-        bean.setApplicationContext(applicationContext);
-
-        configureInterface(attributes, bean);
-
-        configureConsumerConfig(attributes, bean);
-
-        configureMethodConfig(attributes, bean);
-
-        bean.afterPropertiesSet();
-
+    public static ReferenceBeanBuilder create(AnnotationAttributes attributes, ApplicationContext applicationContext) {
+        return new ReferenceBeanBuilder(attributes, applicationContext);
     }
 
-    @Deprecated
-    public static ReferenceBeanBuilder create(Reference reference, ClassLoader classLoader,
-                                              ApplicationContext applicationContext) {
-        return create(fromMap(getAttributes(reference, applicationContext.getEnvironment(), true)), applicationContext);
+    public ReferenceBeanBuilder defaultInterfaceClass(Class<?> interfaceClass) {
+        this.defaultInterfaceClass = interfaceClass;
+        return this;
     }
 
-    public static ReferenceBeanBuilder create(AnnotationAttributes attributes, ApplicationContext applicationContext) {
-        return new ReferenceBeanBuilder(attributes, applicationContext);
-    }
 }
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ServiceClassPostProcessor.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ServiceClassPostProcessor.java
index b1c9a30..f6dbe69 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ServiceClassPostProcessor.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/beans/factory/annotation/ServiceClassPostProcessor.java
@@ -289,12 +289,12 @@ public class ServiceClassPostProcessor implements BeanDefinitionRegistryPostProc
 
         String annotatedServiceBeanName = beanDefinitionHolder.getBeanName();
 
-        AbstractBeanDefinition serviceBeanDefinition =
-                buildServiceBeanDefinition(service, serviceAnnotationAttributes, interfaceClass, annotatedServiceBeanName);
-
         // ServiceBean Bean name
         String beanName = generateServiceBeanName(serviceAnnotationAttributes, interfaceClass);
 
+        AbstractBeanDefinition serviceBeanDefinition =
+                buildServiceBeanDefinition(beanName, service, serviceAnnotationAttributes, interfaceClass, annotatedServiceBeanName);
+
         if (scanner.checkCandidate(beanName, serviceBeanDefinition)) { // check duplicated candidate bean
             registry.registerBeanDefinition(beanName, serviceBeanDefinition);
 
@@ -376,6 +376,8 @@ public class ServiceClassPostProcessor implements BeanDefinitionRegistryPostProc
     /**
      * Build the {@link AbstractBeanDefinition Bean Definition}
      *
+     *
+     * @param beanName
      * @param serviceAnnotation
      * @param serviceAnnotationAttributes
      * @param interfaceClass
@@ -383,7 +385,7 @@ public class ServiceClassPostProcessor implements BeanDefinitionRegistryPostProc
      * @return
      * @since 2.7.3
      */
-    private AbstractBeanDefinition buildServiceBeanDefinition(Annotation serviceAnnotation,
+    private AbstractBeanDefinition buildServiceBeanDefinition(String beanName, Annotation serviceAnnotation,
                                                               AnnotationAttributes serviceAnnotationAttributes,
                                                               Class<?> interfaceClass,
                                                               String annotatedServiceBeanName) {
@@ -399,6 +401,8 @@ public class ServiceClassPostProcessor implements BeanDefinitionRegistryPostProc
 
         propertyValues.addPropertyValues(new AnnotationPropertyValuesAdapter(serviceAnnotation, environment, ignoreAttributeNames));
 
+        //set config id, for ConfigManager cache key
+        builder.addPropertyValue("id", beanName);
         // References "ref" property to annotated-@Service Bean
         addPropertyReference(builder, "ref", annotatedServiceBeanName);
         // Set interface
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/schema/DubboBeanDefinitionParser.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/schema/DubboBeanDefinitionParser.java
index 084479a..70f4dfe 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/schema/DubboBeanDefinitionParser.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/schema/DubboBeanDefinitionParser.java
@@ -26,16 +26,17 @@ import org.apache.dubbo.config.ConsumerConfig;
 import org.apache.dubbo.config.MethodConfig;
 import org.apache.dubbo.config.ProtocolConfig;
 import org.apache.dubbo.config.ProviderConfig;
+import org.apache.dubbo.config.ReferenceConfig;
 import org.apache.dubbo.config.RegistryConfig;
 import org.apache.dubbo.config.spring.ReferenceBean;
 import org.apache.dubbo.config.spring.ServiceBean;
-
 import org.springframework.beans.PropertyValue;
 import org.springframework.beans.factory.config.BeanDefinition;
 import org.springframework.beans.factory.config.BeanDefinitionHolder;
 import org.springframework.beans.factory.config.RuntimeBeanReference;
 import org.springframework.beans.factory.config.TypedStringValue;
 import org.springframework.beans.factory.support.AbstractBeanDefinition;
+import org.springframework.beans.factory.support.GenericBeanDefinition;
 import org.springframework.beans.factory.support.ManagedList;
 import org.springframework.beans.factory.support.ManagedMap;
 import org.springframework.beans.factory.support.RootBeanDefinition;
@@ -50,7 +51,9 @@ import org.w3c.dom.NodeList;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
 import java.util.Date;
+import java.util.HashMap;
 import java.util.HashSet;
+import java.util.Map;
 import java.util.Set;
 import java.util.regex.Pattern;
 
@@ -71,6 +74,7 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
     private static final String METHOD = "Method";
     private final Class<?> beanClass;
     private final boolean required;
+    private static Map<String, Map<String, Class>> beanPropsCache = new HashMap<>();
 
     public DubboBeanDefinitionParser(Class<?> beanClass, boolean required) {
         this.beanClass = beanClass;
@@ -101,6 +105,8 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
                 id = generatedBeanName + (counter++);
             }
         }
+
+        Set<String> processedProps = new HashSet<>();
         if (StringUtils.isNotEmpty(id)) {
             if (parserContext.getRegistry().containsBeanDefinition(id)) {
                 throw new IllegalStateException("Duplicate spring bean id " + id);
@@ -128,13 +134,133 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
                 parseProperties(element.getChildNodes(), classDefinition, parserContext);
                 beanDefinition.getPropertyValues().addPropertyValue("ref", new BeanDefinitionHolder(classDefinition, id + "Impl"));
             }
-        } else if (ProviderConfig.class.equals(beanClass)) {
+
+        }
+
+
+        Map<String, Class> beanPropsMap = beanPropsCache.get(beanClass.getName());
+        if (beanPropsMap == null) {
+            beanPropsMap = new HashMap<>();
+            beanPropsCache.put(beanClass.getName(), beanPropsMap);
+
+            if (ReferenceBean.class.equals(beanClass)) {
+                //extract bean props from ReferenceConfig
+                getPropertyMap(ReferenceConfig.class, beanPropsMap);
+            } else {
+                getPropertyMap(beanClass, beanPropsMap);
+            }
+        }
+
+        ManagedMap parameters = null;
+        for (Map.Entry<String, Class> entry : beanPropsMap.entrySet()) {
+            String beanProperty = entry.getKey();
+            Class type = entry.getValue();
+            String property = StringUtils.camelToSplitName(beanProperty, "-");
+            processedProps.add(property);
+            if ("parameters".equals(property)) {
+                parameters = parseParameters(element.getChildNodes(), beanDefinition, parserContext);
+            } else if ("methods".equals(property)) {
+                parseMethods(id, element.getChildNodes(), beanDefinition, parserContext);
+            } else if ("arguments".equals(property)) {
+                parseArguments(id, element.getChildNodes(), beanDefinition, parserContext);
+            } else {
+                String value = resolveAttribute(element, property, parserContext);
+                if (value != null) {
+                    value = value.trim();
+                    if (value.length() > 0) {
+                        if ("registry".equals(property) && RegistryConfig.NO_AVAILABLE.equalsIgnoreCase(value)) {
+                            RegistryConfig registryConfig = new RegistryConfig();
+                            registryConfig.setAddress(RegistryConfig.NO_AVAILABLE);
+                            beanDefinition.getPropertyValues().addPropertyValue(beanProperty, registryConfig);
+                        } else if ("provider".equals(property) || "registry".equals(property) || ("protocol".equals(property) && AbstractServiceConfig.class.isAssignableFrom(beanClass))) {
+                            /**
+                             * For 'provider' 'protocol' 'registry', keep literal value (should be id/name) and set the value to 'registryIds' 'providerIds' protocolIds'
+                             * The following process should make sure each id refers to the corresponding instance, here's how to find the instance for different use cases:
+                             * 1. Spring, check existing bean by id, see{@link ServiceBean#afterPropertiesSet()}; then try to use id to find configs defined in remote Config Center
+                             * 2. API, directly use id to find configs defined in remote Config Center; if all config instances are defined locally, please use {@link ServiceConfig#setRegistries(List)}
+                             */
+                            beanDefinition.getPropertyValues().addPropertyValue(beanProperty + "Ids", value);
+                        } else {
+                            Object reference;
+                            if (isPrimitive(type)) {
+                                value = getCompatibleDefaultValue(property, value);
+                                reference = value;
+                            } else if (ONRETURN.equals(property) || ONTHROW.equals(property) || ONINVOKE.equals(property)) {
+                                int index = value.lastIndexOf(".");
+                                String ref = value.substring(0, index);
+                                String method = value.substring(index + 1);
+                                reference = new RuntimeBeanReference(ref);
+                                beanDefinition.getPropertyValues().addPropertyValue(property + METHOD, method);
+                            } else {
+                                if ("ref".equals(property) && parserContext.getRegistry().containsBeanDefinition(value)) {
+                                    BeanDefinition refBean = parserContext.getRegistry().getBeanDefinition(value);
+                                    if (!refBean.isSingleton()) {
+                                        throw new IllegalStateException("The exported service ref " + value + " must be singleton! Please set the " + value + " bean scope to singleton, eg: <bean id=\"" + value + "\" scope=\"singleton\" ...>");
+                                    }
+                                }
+                                reference = new RuntimeBeanReference(value);
+                            }
+                            beanDefinition.getPropertyValues().addPropertyValue(beanProperty, reference);
+                        }
+                    }
+                }
+            }
+        }
+
+        NamedNodeMap attributes = element.getAttributes();
+        int len = attributes.getLength();
+        for (int i = 0; i < len; i++) {
+            Node node = attributes.item(i);
+            String name = node.getLocalName();
+            if (!processedProps.contains(name)) {
+                if (parameters == null) {
+                    parameters = new ManagedMap();
+                }
+                String value = node.getNodeValue();
+                parameters.put(name, new TypedStringValue(value, String.class));
+            }
+        }
+        if (parameters != null) {
+            beanDefinition.getPropertyValues().addPropertyValue("parameters", parameters);
+        }
+
+        // post-process after parse attributes
+        if (ProviderConfig.class.equals(beanClass)) {
             parseNested(element, parserContext, ServiceBean.class, true, "service", "provider", id, beanDefinition);
         } else if (ConsumerConfig.class.equals(beanClass)) {
             parseNested(element, parserContext, ReferenceBean.class, false, "reference", "consumer", id, beanDefinition);
+        } else if (ReferenceBean.class.equals(beanClass)) {
+            configReferenceBean(element, parserContext, beanDefinition, null);
         }
-        Set<String> props = new HashSet<>();
-        ManagedMap parameters = null;
+
+        return beanDefinition;
+    }
+
+    private static void configReferenceBean(Element element, ParserContext parserContext, RootBeanDefinition beanDefinition, BeanDefinition consumerDefinition) {
+        // process interface class
+        String interfaceClassName = resolveAttribute(element, "interface", parserContext);
+        String generic = resolveAttribute(element, "generic", parserContext);
+        if (StringUtils.isBlank(generic) && consumerDefinition != null) {
+            // get generic from consumerConfig
+            generic = (String) consumerDefinition.getPropertyValues().get("generic");
+        }
+        if (generic != null) {
+            Environment environment = parserContext.getReaderContext().getEnvironment();
+            generic = environment.resolvePlaceholders(generic);
+            beanDefinition.getPropertyValues().add("generic", generic);
+        }
+
+        Class interfaceClass = ReferenceConfig.determineInterfaceClass(generic, interfaceClassName);
+
+        // create decorated definition for reference bean, Avoid being instantiated when getting the beanType of ReferenceBean
+        // refer to org.springframework.beans.factory.support.AbstractBeanFactory#getType()
+        GenericBeanDefinition targetDefinition = new GenericBeanDefinition();
+        targetDefinition.setBeanClass(interfaceClass);
+        String id = (String) beanDefinition.getPropertyValues().get("id");
+        beanDefinition.setDecoratedDefinition(new BeanDefinitionHolder(targetDefinition, id+"_decorated"));
+    }
+
+    private static void getPropertyMap(Class<?> beanClass, Map<String, Class> beanPropsMap) {
         for (Method setter : beanClass.getMethods()) {
             String name = setter.getName();
             if (name.length() > 3 && name.startsWith("set")
@@ -142,8 +268,6 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
                     && setter.getParameterTypes().length == 1) {
                 Class<?> type = setter.getParameterTypes()[0];
                 String beanProperty = name.substring(3, 4).toLowerCase() + name.substring(4);
-                String property = StringUtils.camelToSplitName(beanProperty, "-");
-                props.add(property);
                 // check the setter/getter whether match
                 Method getter = null;
                 try {
@@ -161,81 +285,22 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
                         || !type.equals(getter.getReturnType())) {
                     continue;
                 }
-                if ("parameters".equals(property)) {
-                    parameters = parseParameters(element.getChildNodes(), beanDefinition, parserContext);
-                } else if ("methods".equals(property)) {
-                    parseMethods(id, element.getChildNodes(), beanDefinition, parserContext);
-                } else if ("arguments".equals(property)) {
-                    parseArguments(id, element.getChildNodes(), beanDefinition, parserContext);
-                } else {
-                    String value = resolveAttribute(element, property, parserContext);
-                    if (value != null) {
-                        value = value.trim();
-                        if (value.length() > 0) {
-                            if ("registry".equals(property) && RegistryConfig.NO_AVAILABLE.equalsIgnoreCase(value)) {
-                                RegistryConfig registryConfig = new RegistryConfig();
-                                registryConfig.setAddress(RegistryConfig.NO_AVAILABLE);
-                                beanDefinition.getPropertyValues().addPropertyValue(beanProperty, registryConfig);
-                            } else if ("provider".equals(property) || "registry".equals(property) || ("protocol".equals(property) && AbstractServiceConfig.class.isAssignableFrom(beanClass))) {
-                                /**
-                                 * For 'provider' 'protocol' 'registry', keep literal value (should be id/name) and set the value to 'registryIds' 'providerIds' protocolIds'
-                                 * The following process should make sure each id refers to the corresponding instance, here's how to find the instance for different use cases:
-                                 * 1. Spring, check existing bean by id, see{@link ServiceBean#afterPropertiesSet()}; then try to use id to find configs defined in remote Config Center
-                                 * 2. API, directly use id to find configs defined in remote Config Center; if all config instances are defined locally, please use {@link ServiceConfig#setRegistries(List)}
-                                 */
-                                beanDefinition.getPropertyValues().addPropertyValue(beanProperty + "Ids", value);
-                            } else {
-                                Object reference;
-                                if (isPrimitive(type)) {
-                                    if ("async".equals(property) && "false".equals(value)
-                                            || "timeout".equals(property) && "0".equals(value)
-                                            || "delay".equals(property) && "0".equals(value)
-                                            || "version".equals(property) && "0.0.0".equals(value)
-                                            || "stat".equals(property) && "-1".equals(value)
-                                            || "reliable".equals(property) && "false".equals(value)) {
-                                        // backward compatibility for the default value in old version's xsd
-                                        value = null;
-                                    }
-                                    reference = value;
-                                } else if (ONRETURN.equals(property) || ONTHROW.equals(property) || ONINVOKE.equals(property)) {
-                                    int index = value.lastIndexOf(".");
-                                    String ref = value.substring(0, index);
-                                    String method = value.substring(index + 1);
-                                    reference = new RuntimeBeanReference(ref);
-                                    beanDefinition.getPropertyValues().addPropertyValue(property + METHOD, method);
-                                } else {
-                                    if ("ref".equals(property) && parserContext.getRegistry().containsBeanDefinition(value)) {
-                                        BeanDefinition refBean = parserContext.getRegistry().getBeanDefinition(value);
-                                        if (!refBean.isSingleton()) {
-                                            throw new IllegalStateException("The exported service ref " + value + " must be singleton! Please set the " + value + " bean scope to singleton, eg: <bean id=\"" + value + "\" scope=\"singleton\" ...>");
-                                        }
-                                    }
-                                    reference = new RuntimeBeanReference(value);
-                                }
-                                beanDefinition.getPropertyValues().addPropertyValue(beanProperty, reference);
-                            }
-                        }
-                    }
-                }
+                beanPropsMap.put(beanProperty, type);
             }
         }
-        NamedNodeMap attributes = element.getAttributes();
-        int len = attributes.getLength();
-        for (int i = 0; i < len; i++) {
-            Node node = attributes.item(i);
-            String name = node.getLocalName();
-            if (!props.contains(name)) {
-                if (parameters == null) {
-                    parameters = new ManagedMap();
-                }
-                String value = node.getNodeValue();
-                parameters.put(name, new TypedStringValue(value, String.class));
-            }
-        }
-        if (parameters != null) {
-            beanDefinition.getPropertyValues().addPropertyValue("parameters", parameters);
+    }
+
+    private static String getCompatibleDefaultValue(String property, String value) {
+        if ("async".equals(property) && "false".equals(value)
+                || "timeout".equals(property) && "0".equals(value)
+                || "delay".equals(property) && "0".equals(value)
+                || "version".equals(property) && "0.0.0".equals(value)
+                || "stat".equals(property) && "-1".equals(value)
+                || "reliable".equals(property) && "false".equals(value)) {
+            // backward compatibility for the default value in old version's xsd
+            value = null;
         }
-        return beanDefinition;
+        return value;
     }
 
     private static boolean isPrimitive(Class<?> cls) {
@@ -265,9 +330,14 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
                         beanDefinition.getPropertyValues().addPropertyValue("default", "false");
                     }
                 }
-                BeanDefinition subDefinition = parse((Element) node, parserContext, beanClass, required);
-                if (subDefinition != null && StringUtils.isNotEmpty(ref)) {
-                    subDefinition.getPropertyValues().addPropertyValue(property, new RuntimeBeanReference(ref));
+                RootBeanDefinition subDefinition = parse((Element) node, parserContext, beanClass, required);
+                if (subDefinition != null) {
+                    if (StringUtils.isNotEmpty(ref)) {
+                        subDefinition.getPropertyValues().addPropertyValue(property, new RuntimeBeanReference(ref));
+                    }
+                    if (ReferenceBean.class.equals(beanClass)) {
+                        configReferenceBean((Element) node, parserContext, subDefinition, beanDefinition);
+                    }
                 }
             }
         }
@@ -416,7 +486,10 @@ public class DubboBeanDefinitionParser implements BeanDefinitionParser {
 
     private static String resolveAttribute(Element element, String attributeName, ParserContext parserContext) {
         String attributeValue = element.getAttribute(attributeName);
-        Environment environment = parserContext.getReaderContext().getEnvironment();
-        return environment.resolvePlaceholders(attributeValue);
+        //https://github.com/apache/dubbo/pull/6079
+        //https://github.com/apache/dubbo/issues/6035
+//        Environment environment = parserContext.getReaderContext().getEnvironment();
+//        return environment.resolvePlaceholders(attributeValue);
+        return attributeValue;
     }
 }
diff --git a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/util/DubboBeanUtils.java b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/util/DubboBeanUtils.java
index 3b2c31c..cb2fa00 100644
--- a/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/util/DubboBeanUtils.java
+++ b/dubbo-config/dubbo-config-spring/src/main/java/org/apache/dubbo/config/spring/util/DubboBeanUtils.java
@@ -16,15 +16,25 @@
  */
 package org.apache.dubbo.config.spring.util;
 
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.dubbo.config.spring.DubboConfigInitializationPostProcessor;
+import org.apache.dubbo.config.spring.ReferenceBeanManager;
 import org.apache.dubbo.config.spring.beans.factory.annotation.DubboConfigAliasPostProcessor;
 import org.apache.dubbo.config.spring.beans.factory.annotation.ReferenceAnnotationBeanPostProcessor;
 import org.apache.dubbo.config.spring.beans.factory.config.DubboConfigDefaultPropertyValueBeanPostProcessor;
 import org.apache.dubbo.config.spring.context.DubboBootstrapApplicationListener;
 import org.apache.dubbo.config.spring.context.DubboLifecycleComponentApplicationListener;
-
+import org.springframework.beans.factory.config.BeanDefinition;
+import org.springframework.beans.factory.config.PropertyPlaceholderConfigurer;
+import org.springframework.beans.factory.support.BeanDefinitionBuilder;
 import org.springframework.beans.factory.support.BeanDefinitionRegistry;
+import org.springframework.beans.factory.support.RootBeanDefinition;
+import org.springframework.context.support.PropertySourcesPlaceholderConfigurer;
 
-import static com.alibaba.spring.util.BeanRegistrar.registerInfrastructureBean;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
 
 /**
  * Dubbo Bean utilities class
@@ -33,6 +43,8 @@ import static com.alibaba.spring.util.BeanRegistrar.registerInfrastructureBean;
  */
 public interface DubboBeanUtils {
 
+    static final Log log = LogFactory.getLog(DubboBeanUtils.class);
+
     /**
      * Register the common beans
      *
@@ -45,6 +57,8 @@ public interface DubboBeanUtils {
      */
     static void registerCommonBeans(BeanDefinitionRegistry registry) {
 
+        registerInfrastructureBean(registry, ReferenceBeanManager.BEAN_NAME, ReferenceBeanManager.class);
+
         // Since 2.5.7 Register @Reference Annotation Bean Processor as an infrastructure Bean
         registerInfrastructureBean(registry, ReferenceAnnotationBeanPostProcessor.BEAN_NAME,
                 ReferenceAnnotationBeanPostProcessor.class);
@@ -64,5 +78,90 @@ public interface DubboBeanUtils {
         // Since 2.7.6 Register DubboConfigDefaultPropertyValueBeanPostProcessor as an infrastructure Bean
         registerInfrastructureBean(registry, DubboConfigDefaultPropertyValueBeanPostProcessor.BEAN_NAME,
                 DubboConfigDefaultPropertyValueBeanPostProcessor.class);
+
+        registerInfrastructureBean(registry, DubboConfigInitializationPostProcessor.BEAN_NAME, DubboConfigInitializationPostProcessor.class);
     }
+
+    /**
+     * Register Infrastructure Bean
+     *
+     * @param beanDefinitionRegistry {@link BeanDefinitionRegistry}
+     * @param beanType               the type of bean
+     * @param beanName               the name of bean
+     * @return if it's a first time to register, return <code>true</code>, or <code>false</code>
+     */
+    static boolean registerInfrastructureBean(BeanDefinitionRegistry beanDefinitionRegistry,
+                                                     String beanName,
+                                                     Class<?> beanType) {
+
+        boolean registered = false;
+
+        if (!beanDefinitionRegistry.containsBeanDefinition(beanName)) {
+            RootBeanDefinition beanDefinition = new RootBeanDefinition(beanType);
+            beanDefinition.setRole(BeanDefinition.ROLE_INFRASTRUCTURE);
+            beanDefinitionRegistry.registerBeanDefinition(beanName, beanDefinition);
+            registered = true;
+
+            if (log.isDebugEnabled()) {
+                log.debug("The Infrastructure bean definition [" + beanDefinition
+                        + "with name [" + beanName + "] has been registered.");
+            }
+        }
+
+        return registered;
+    }
+
+    /**
+     * Call this method in postProcessBeanFactory()
+     *
+     * @param registry
+     */
+    static void registerBeansIfNotExists(BeanDefinitionRegistry registry) {
+        // Resolve ${...} placeholders of bean definition with Spring Environment
+
+        // If PropertyPlaceholderConfigurer already exists, PropertySourcesPlaceholderConfigurer cannot be registered.
+        // When both of them exist, a conflict will occur, and an exception will be thrown when encountering an unresolved placeholder
+
+        if (!checkBeanExists(registry, PropertyPlaceholderConfigurer.class)) {
+            Map<String, Object> propertySourcesPlaceholderPropertyValues = new HashMap<>();
+            // to make sure the default PropertySourcesPlaceholderConfigurer's priority is higher than PropertyPlaceholderConfigurer
+            propertySourcesPlaceholderPropertyValues.put("order", 0);
+            registerBeanDefinitionIfNotExists(registry, PropertySourcesPlaceholderConfigurer.class.getName(),
+                    PropertySourcesPlaceholderConfigurer.class, propertySourcesPlaceholderPropertyValues);
+        }
+    }
+
+    static boolean registerBeanDefinitionIfNotExists(BeanDefinitionRegistry registry, String beanName,
+                                                     Class<?> beanClass, Map<String, Object> extraPropertyValues) {
+        if (registry.containsBeanDefinition(beanName)) {
+            return false;
+        }
+
+        if (checkBeanExists(registry, beanClass)) {
+            return false;
+        }
+
+        BeanDefinition beanDefinition = BeanDefinitionBuilder.genericBeanDefinition(beanClass).getBeanDefinition();
+        if (extraPropertyValues != null) {
+            for (Map.Entry<String, Object> entry : extraPropertyValues.entrySet()) {
+                beanDefinition.getPropertyValues().add(entry.getKey(), entry.getValue());
+            }
+        }
+
+        registry.registerBeanDefinition(beanName, beanDefinition);
+
+        return true;
+    }
+
+    static boolean checkBeanExists(BeanDefinitionRegistry registry, Class<?> beanClass) {
+        String[] candidates = registry.getBeanDefinitionNames();
+        for (String candidate : candidates) {
+            BeanDefinition beanDefinition = registry.getBeanDefinition(candidate);
+            if (Objects.equals(beanDefinition.getBeanClassName(), beanClass.getName())) {
+                return true;
+            }
+        }
+        return false;
+    }
+
 }
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/ConfigTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/ConfigTest.java
index 6269258..882a9f5 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/ConfigTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/ConfigTest.java
@@ -45,11 +45,14 @@ import org.apache.dubbo.rpc.Exporter;
 import org.apache.dubbo.rpc.Filter;
 import org.apache.dubbo.rpc.RpcContext;
 import org.apache.dubbo.rpc.RpcException;
+import org.apache.dubbo.rpc.model.ApplicationModel;
 import org.apache.dubbo.rpc.service.GenericService;
 
 import org.junit.Assert;
 import org.junit.Ignore;
 import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.springframework.beans.factory.BeanCreationException;
 import org.springframework.context.annotation.AnnotationConfigApplicationContext;
 import org.springframework.context.support.ClassPathXmlApplicationContext;
@@ -74,9 +77,22 @@ import static org.junit.matchers.JUnitMatchers.containsString;
 @Ignore
 public class ConfigTest {
 
+    private static String resourcePath = ConfigTest.class.getPackage().getName().replace('.', '/');
+
+    @BeforeEach
+    public void setUp() {
+        ApplicationModel.reset();
+    }
+
+    @AfterEach
+    public void tearDown() {
+        ApplicationModel.reset();
+    }
+
+
     @Test
     public void testSpringExtensionInject() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/spring-extension-inject.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/spring-extension-inject.xml");
         ctx.start();
         try {
             MockFilter filter = (MockFilter) ExtensionLoader.getExtensionLoader(Filter.class).getExtension("mymock");
@@ -91,7 +107,7 @@ public class ConfigTest {
 
     @Test
     public void testServiceClass() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/service-class.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/service-class.xml");
         ctx.start();
         try {
             DemoService demoService = refer("dubbo://127.0.0.1:30887");
@@ -129,7 +145,7 @@ public class ConfigTest {
     @Test
     @SuppressWarnings("unchecked")
     public void testProviderNestedService() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/provider-nested-service.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/provider-nested-service.xml");
         ctx.start();
         try {
             ServiceConfig<DemoService> serviceConfig = (ServiceConfig<DemoService>) ctx.getBean("serviceConfig");
@@ -188,7 +204,7 @@ public class ConfigTest {
 
     @Test
     public void testMultiProtocol() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-protocol.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-protocol.xml");
         ctx.start();
         try {
             DemoService demoService = refer("dubbo://127.0.0.1:20881");
@@ -202,7 +218,7 @@ public class ConfigTest {
 
     @Test
     public void testMultiProtocolDefault() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-protocol-default.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-protocol-default.xml");
         ctx.start();
         try {
             DemoService demoService = refer("rmi://127.0.0.1:10991");
@@ -217,7 +233,7 @@ public class ConfigTest {
     @Test
     public void testMultiProtocolError() {
         try {
-            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-protocol-error.xml");
+            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-protocol-error.xml");
             ctx.start();
             ctx.stop();
             ctx.close();
@@ -230,7 +246,7 @@ public class ConfigTest {
     public void testMultiProtocolRegister() {
         SimpleRegistryService registryService = new SimpleRegistryService();
         Exporter<RegistryService> exporter = SimpleRegistryExporter.export(4547, registryService);
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-protocol-register.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-protocol-register.xml");
         ctx.start();
         try {
             List<URL> urls = registryService.getRegistered().get("org.apache.dubbo.config.spring.api.DemoService");
@@ -250,7 +266,7 @@ public class ConfigTest {
         Exporter<RegistryService> exporter1 = SimpleRegistryExporter.export(4545, registryService1);
         SimpleRegistryService registryService2 = new SimpleRegistryService();
         Exporter<RegistryService> exporter2 = SimpleRegistryExporter.export(4546, registryService2);
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-registry.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-registry.xml");
         ctx.start();
         try {
             List<URL> urls1 = registryService1.getRegistered().get("org.apache.dubbo.config.spring.api.DemoService");
@@ -271,7 +287,7 @@ public class ConfigTest {
     public void testDelayFixedTime() throws Exception {
         SimpleRegistryService registryService = new SimpleRegistryService();
         Exporter<RegistryService> exporter = SimpleRegistryExporter.export(4548, registryService);
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/delay-fixed-time.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/delay-fixed-time.xml");
         ctx.start();
         try {
             List<URL> urls = registryService.getRegistered().get("org.apache.dubbo.config.spring.api.DemoService");
@@ -295,7 +311,7 @@ public class ConfigTest {
     public void testDelayOnInitialized() throws Exception {
         SimpleRegistryService registryService = new SimpleRegistryService();
         Exporter<RegistryService> exporter = SimpleRegistryExporter.export(4548, registryService);
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/delay-on-initialized.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/delay-on-initialized.xml");
         //ctx.start();
         try {
             List<URL> urls = registryService.getRegistered().get("org.apache.dubbo.config.spring.api.DemoService");
@@ -323,10 +339,12 @@ public class ConfigTest {
 
     @Test
     public void testAutowireAndAOP() throws Exception {
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(
+                resourcePath + "/demo-provider.xml",
+                resourcePath + "/demo-provider-properties.xml");
         providerContext.start();
         try {
-            ClassPathXmlApplicationContext byNameContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/aop-autowire-byname.xml");
+            ClassPathXmlApplicationContext byNameContext = new ClassPathXmlApplicationContext(resourcePath + "/aop-autowire-byname.xml");
             byNameContext.start();
             try {
                 DemoActionBySetter demoActionBySetter = (DemoActionBySetter) byNameContext.getBean("demoActionBySetter");
@@ -339,7 +357,7 @@ public class ConfigTest {
                 byNameContext.stop();
                 byNameContext.close();
             }
-            ClassPathXmlApplicationContext byTypeContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/aop-autowire-bytype.xml");
+            ClassPathXmlApplicationContext byTypeContext = new ClassPathXmlApplicationContext(resourcePath + "/aop-autowire-bytype.xml");
             byTypeContext.start();
             try {
                 DemoActionBySetter demoActionBySetter = (DemoActionBySetter) byTypeContext.getBean("demoActionBySetter");
@@ -408,14 +426,23 @@ public class ConfigTest {
 
     @Test
     public void testInitReference() throws Exception {
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(
+                resourcePath + "/demo-provider.xml",
+                resourcePath + "/demo-provider-properties.xml");
         providerContext.start();
         try {
-            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/init-reference.xml");
+            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/init-reference.xml",
+                    resourcePath + "/init-reference-properties.xml");
             ctx.start();
             try {
                 DemoService demoService = (DemoService) ctx.getBean("demoService");
                 assertEquals("say:world", demoService.sayName("world"));
+
+                GenericService demoService2 = (GenericService) ctx.getBean("demoService2");
+                assertEquals("say:world", demoService2.$invoke("sayName", new String[]{"java.lang.String"}, new Object[]{"world"}));
+
+            } catch (Throwable ex){
+                ex.printStackTrace();
             } finally {
                 ctx.stop();
                 ctx.close();
@@ -429,7 +456,7 @@ public class ConfigTest {
     // DUBBO-571 methods key in provider's URLONE doesn't contain the methods from inherited super interface
     @Test
     public void test_noMethodInterface_methodsKeyHasValue() throws Exception {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider-no-methods-interface.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/demo-provider-no-methods-interface.xml");
         ctx.start();
         try {
             ServiceBean bean = (ServiceBean) ctx.getBean("service");
@@ -444,16 +471,14 @@ public class ConfigTest {
     }
 
     // DUBBO-147 find all invoker instances which have been tried from RpcContext
-    @Test
+    //@Test
     public void test_RpcContext_getUrls() throws Exception {
         ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(
-                ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider-long-waiting.xml");
+                resourcePath + "/demo-provider-long-waiting.xml");
         providerContext.start();
 
         try {
-            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(
-                    ConfigTest.class.getPackage().getName().replace('.', '/')
-                            + "/init-reference-getUrls.xml");
+            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/init-reference-getUrls.xml");
             ctx.start();
             try {
                 DemoService demoService = (DemoService) ctx.getBean("demoService");
@@ -478,14 +503,12 @@ public class ConfigTest {
     // BUG: DUBBO-846 in version 2.0.9, config retry="false" on provider's method doesn't work
     @Test
     public void test_retrySettingFail() throws Exception {
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(
-                ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider-long-waiting.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/demo-provider-long-waiting.xml");
         providerContext.start();
 
         try {
             ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(
-                    ConfigTest.class.getPackage().getName().replace('.', '/')
-                            + "/init-reference-retry-false.xml");
+                    resourcePath + "/init-reference-retry-false.xml");
             ctx.start();
             try {
                 DemoService demoService = (DemoService) ctx.getBean("demoService");
@@ -511,10 +534,11 @@ public class ConfigTest {
     // for example, object transported on the wire doesn't implement Serializable
     @Test
     public void test_returnSerializationFail() throws Exception {
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider-UnserializableBox.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/demo-provider-UnserializableBox.xml");
         providerContext.start();
         try {
-            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/init-reference.xml");
+            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/init-reference.xml",
+                    resourcePath + "/init-reference-properties.xml");
             ctx.start();
             try {
                 DemoService demoService = (DemoService) ctx.getBean("demoService");
@@ -536,7 +560,7 @@ public class ConfigTest {
 
     @Test
     public void testXmlOverrideProperties() throws Exception {
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/xml-override-properties.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/xml-override-properties.xml");
         providerContext.start();
         try {
             ApplicationConfig application = (ApplicationConfig) providerContext.getBean("application");
@@ -602,7 +626,7 @@ public class ConfigTest {
     @Test
     public void testSystemPropertyOverrideProtocol() throws Exception {
         System.setProperty("dubbo.protocol.port", "20812");
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/override-protocol.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/override-protocol.xml");
         providerContext.start();
         try {
             ProtocolConfig dubbo = (ProtocolConfig) providerContext.getBean("dubbo");
@@ -618,7 +642,7 @@ public class ConfigTest {
     public void testSystemPropertyOverrideMultiProtocol() throws Exception {
         System.setProperty("dubbo.protocol.dubbo.port", "20814");
         System.setProperty("dubbo.protocol.rmi.port", "10914");
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/override-multi-protocol.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/override-multi-protocol.xml");
         providerContext.start();
         try {
             ProtocolConfig dubbo = (ProtocolConfig) providerContext.getBean("dubbo");
@@ -641,7 +665,7 @@ public class ConfigTest {
         System.setProperty("dubbo.registry.address", "N/A");
         System.setProperty("dubbo.protocol.name", "dubbo");
         System.setProperty("dubbo.protocol.port", "20819");
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/system-properties-override-default.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/system-properties-override-default.xml");
         providerContext.start();
         try {
             ServiceConfig<DemoService> service = (ServiceConfig<DemoService>) providerContext.getBean("demoServiceConfig");
@@ -670,7 +694,7 @@ public class ConfigTest {
         System.setProperty("dubbo.protocol.name", "dubbo");
         System.setProperty("dubbo.protocol.port", "20819");
         System.setProperty("dubbo.service.register", "false");
-        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/system-properties-override.xml");
+        ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/system-properties-override.xml");
         providerContext.start();
         try {
             ServiceConfig<DemoService> service = (ServiceConfig<DemoService>) providerContext.getBean("demoServiceConfig");
@@ -857,7 +881,7 @@ public class ConfigTest {
     @SuppressWarnings("unchecked")
     public void testCustomizeParameter() throws Exception {
         ClassPathXmlApplicationContext context =
-                new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/customize-parameter.xml");
+                new ClassPathXmlApplicationContext(resourcePath + "/customize-parameter.xml");
         context.start();
         ServiceBean<DemoService> serviceBean = (ServiceBean<DemoService>) context.getBean("demoServiceExport");
         URL url = (URL) serviceBean.getExportedUrls().get(0);
@@ -882,10 +906,10 @@ public class ConfigTest {
         SimpleRegistryService registryService = new SimpleRegistryService();
         Exporter<RegistryService> exporter = SimpleRegistryExporter.export(4548, registryService);
         try {
-            ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/annotation-provider.xml");
+            ClassPathXmlApplicationContext providerContext = new ClassPathXmlApplicationContext(resourcePath + "/annotation-provider.xml");
             providerContext.start();
             try {
-                ClassPathXmlApplicationContext consumerContext = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/annotation-consumer.xml");
+                ClassPathXmlApplicationContext consumerContext = new ClassPathXmlApplicationContext(resourcePath + "/annotation-consumer.xml");
                 consumerContext.start();
                 try {
                     AnnotationAction annotationAction = (AnnotationAction) consumerContext.getBean("annotationAction");
@@ -1044,7 +1068,7 @@ public class ConfigTest {
 
     @Test
     public void testGenericServiceConfigThroughSpring() throws Exception {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/generic-export.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/generic-export.xml");
         try {
             ctx.start();
             ServiceConfig serviceConfig = (ServiceConfig) ctx.getBean("dubboDemoService");
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapterTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapterTest.java
index d51fe3c..dfcf9aa 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapterTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/AnnotationPropertyValuesAdapterTest.java
@@ -18,12 +18,13 @@ package org.apache.dubbo.config.spring.beans.factory.annotation;
 
 
 import org.apache.dubbo.common.utils.CollectionUtils;
+import org.apache.dubbo.config.ReferenceConfig;
 import org.apache.dubbo.config.annotation.Reference;
-import org.apache.dubbo.config.spring.ReferenceBean;
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.apache.dubbo.config.spring.api.DemoService;
-
 import org.junit.Assert;
 import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
 import org.springframework.core.annotation.AnnotationUtils;
 import org.springframework.core.convert.converter.Converter;
 import org.springframework.core.convert.support.DefaultConversionService;
@@ -44,6 +45,11 @@ import static org.springframework.util.StringUtils.arrayToCommaDelimitedString;
  */
 public class AnnotationPropertyValuesAdapterTest {
 
+    @BeforeEach
+    public void setUp() {
+        DubboBootstrap.reset();
+    }
+
     @Test
     public void test() {
 
@@ -59,7 +65,7 @@ public class AnnotationPropertyValuesAdapterTest {
 
         AnnotationPropertyValuesAdapter propertyValues = new AnnotationPropertyValuesAdapter(reference, mockEnvironment);
 
-        ReferenceBean referenceBean = new ReferenceBean();
+        ReferenceConfig referenceBean = new ReferenceConfig();
 
         DataBinder dataBinder = new DataBinder(referenceBean);
 
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessorTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessorTest.java
index 598933d..ef9e540 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessorTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceAnnotationBeanPostProcessorTest.java
@@ -16,21 +16,22 @@
  */
 package org.apache.dubbo.config.spring.beans.factory.annotation;
 
+import org.apache.dubbo.config.ReferenceConfig;
 import org.apache.dubbo.config.annotation.Method;
 import org.apache.dubbo.config.annotation.Reference;
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.apache.dubbo.config.spring.ReferenceBean;
+import org.apache.dubbo.config.spring.ReferenceBeanManager;
 import org.apache.dubbo.config.spring.api.DemoService;
 import org.apache.dubbo.config.spring.api.HelloService;
 import org.apache.dubbo.config.utils.ReferenceConfigCache;
-import org.apache.dubbo.rpc.model.ApplicationModel;
-
 import org.aspectj.lang.ProceedingJoinPoint;
 import org.aspectj.lang.annotation.Around;
 import org.aspectj.lang.annotation.Aspect;
-import org.junit.After;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.runner.RunWith;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.beans.factory.annotation.InjectionMetadata;
@@ -44,7 +45,9 @@ import org.springframework.test.context.ContextConfiguration;
 import org.springframework.test.context.TestPropertySource;
 import org.springframework.test.context.junit4.SpringRunner;
 
+import java.lang.reflect.Field;
 import java.util.Collection;
+import java.util.HashMap;
 import java.util.Map;
 
 import static org.apache.dubbo.config.spring.beans.factory.annotation.ReferenceAnnotationBeanPostProcessor.BEAN_NAME;
@@ -70,14 +73,13 @@ import static org.junit.Assert.assertTrue;
 @EnableAspectJAutoProxy(proxyTargetClass = true, exposeProxy = true)
 public class ReferenceAnnotationBeanPostProcessorTest {
 
-    @Before
+    @BeforeEach
     public void setUp() {
-        ApplicationModel.reset();
+        DubboBootstrap.reset();
     }
 
-    @After
+    @AfterEach
     public void tearDown() {
-        ApplicationModel.reset();
     }
 
     private static final String AOP_SUFFIX = "(based on AOP)";
@@ -97,6 +99,11 @@ public class ReferenceAnnotationBeanPostProcessorTest {
         return new TestBean();
     }
 
+    @Bean(ReferenceBeanManager.BEAN_NAME)
+    public ReferenceBeanManager referenceBeanManager() {
+        return new ReferenceBeanManager();
+    }
+
     @Bean(BEAN_NAME)
     public ReferenceAnnotationBeanPostProcessor referenceAnnotationBeanPostProcessor() {
         return new ReferenceAnnotationBeanPostProcessor();
@@ -122,7 +129,7 @@ public class ReferenceAnnotationBeanPostProcessorTest {
     private HelloService helloService2;
 
     @Test
-    public void test() throws Exception {
+    public void testAop() throws Exception {
 
         assertTrue(context.containsBean("helloService"));
 
@@ -135,7 +142,10 @@ public class ReferenceAnnotationBeanPostProcessorTest {
         Assert.assertNotNull(testBean.getDemoServiceFromParent());
         Assert.assertNotNull(testBean.getDemoService());
         Assert.assertNotNull(testBean.autowiredDemoService);
-        Assert.assertEquals(1, demoServicesMap.size());
+        Assert.assertEquals(3, demoServicesMap.size());
+        Assert.assertNotNull(demoServicesMap.get("demoServiceImpl"));
+        Assert.assertNotNull(demoServicesMap.get("my-reference-bean"));
+        Assert.assertNotNull(demoServicesMap.get("@Reference(url=dubbo://127.0.0.1:12345?version=2.5.7,version=2.5.7) org.apache.dubbo.config.spring.api.DemoService"));
 
         String expectedResult = "Hello,Mercy" + AOP_SUFFIX;
 
@@ -154,33 +164,6 @@ public class ReferenceAnnotationBeanPostProcessorTest {
 
         Assert.assertEquals(expectedResult, myDemoService.sayName("Mercy"));
 
-
-        for (DemoService demoService1 : demoServicesMap.values()) {
-
-            Assert.assertEquals(myDemoService, demoService1);
-
-            Assert.assertEquals(expectedResult, demoService1.sayName("Mercy"));
-        }
-
-    }
-
-    /**
-     * Test on {@link ReferenceAnnotationBeanPostProcessor#getReferenceBeans()}
-     */
-    @Test
-    public void testGetReferenceBeans() {
-
-        ReferenceAnnotationBeanPostProcessor beanPostProcessor = context.getBean(BEAN_NAME,
-                ReferenceAnnotationBeanPostProcessor.class);
-
-        Collection<ReferenceBean<?>> referenceBeans = beanPostProcessor.getReferenceBeans();
-
-        Assert.assertEquals(4, referenceBeans.size());
-
-        ReferenceBean<?> referenceBean = referenceBeans.iterator().next();
-
-        Assert.assertNotNull(ReferenceConfigCache.getCache().get(referenceBean));
-
     }
 
     @Test
@@ -194,15 +177,24 @@ public class ReferenceAnnotationBeanPostProcessorTest {
 
         Assert.assertEquals(3, referenceBeanMap.size());
 
+        Map<String, Integer> checkingFieldNames = new HashMap<>();
+        checkingFieldNames.put("helloService", 0);
+        checkingFieldNames.put("helloService2", 0);
+        checkingFieldNames.put("demoServiceFromParent", 0);
+
         for (Map.Entry<InjectionMetadata.InjectedElement, ReferenceBean<?>> entry : referenceBeanMap.entrySet()) {
 
             InjectionMetadata.InjectedElement injectedElement = entry.getKey();
-
-            Assert.assertEquals("com.alibaba.spring.beans.factory.annotation.AbstractAnnotationBeanPostProcessor$AnnotatedFieldElement",
-                    injectedElement.getClass().getName());
-
+            Field field = (Field) injectedElement.getMember();
+            Integer count = checkingFieldNames.get(field.getName());
+            Assert.assertNotNull(count);
+            Assert.assertEquals(0, count.intValue());
+            checkingFieldNames.put(field.getName(), count+1);
         }
 
+        for (Map.Entry<String, Integer> entry : checkingFieldNames.entrySet()) {
+            Assert.assertEquals("check field element failed: "+entry.getKey(), 1, entry.getValue().intValue());
+        }
     }
 
     @Test
@@ -216,17 +208,26 @@ public class ReferenceAnnotationBeanPostProcessorTest {
 
         Assert.assertEquals(2, referenceBeanMap.size());
 
+        Map<String, Integer> checkingMethodNames = new HashMap<>();
+        checkingMethodNames.put("setDemoServiceFromAncestor", 0);
+        checkingMethodNames.put("setDemoService", 0);
+
         for (Map.Entry<InjectionMetadata.InjectedElement, ReferenceBean<?>> entry : referenceBeanMap.entrySet()) {
 
             InjectionMetadata.InjectedElement injectedElement = entry.getKey();
-
-            Assert.assertEquals("com.alibaba.spring.beans.factory.annotation.AbstractAnnotationBeanPostProcessor$AnnotatedMethodElement",
-                    injectedElement.getClass().getName());
-
+            java.lang.reflect.Method method = (java.lang.reflect.Method) injectedElement.getMember();
+            Integer count = checkingMethodNames.get(method.getName());
+            Assert.assertNotNull(count);
+            Assert.assertEquals(0, count.intValue());
+            checkingMethodNames.put(method.getName(), count+1);
         }
 
+        for (Map.Entry<String, Integer> entry : checkingMethodNames.entrySet()) {
+            Assert.assertEquals("check method element failed: "+entry.getKey(), 1, entry.getValue().intValue());
+        }
     }
 
+
     //    @Test
     //    public void testModuleInfo() {
     //
@@ -304,17 +305,23 @@ public class ReferenceAnnotationBeanPostProcessorTest {
     @Test
     public void testReferenceBeansMethodAnnotation() {
 
-        ReferenceAnnotationBeanPostProcessor beanPostProcessor = context.getBean(BEAN_NAME,
-                ReferenceAnnotationBeanPostProcessor.class);
+        ReferenceBeanManager referenceBeanManager = context.getBean(ReferenceBeanManager.BEAN_NAME,
+                ReferenceBeanManager.class);
 
-        Collection<ReferenceBean<?>> referenceBeans = beanPostProcessor.getReferenceBeans();
+        Collection<ReferenceBean> referenceBeans = referenceBeanManager.getReferences();
 
         Assert.assertEquals(4, referenceBeans.size());
 
-        ReferenceBean<?> referenceBean = referenceBeans.iterator().next();
+        for (ReferenceBean referenceBean : referenceBeans) {
+            ReferenceConfig referenceConfig = referenceBean.getReferenceConfig();
+            Assert.assertNotNull(referenceConfig);
+            Assert.assertNotNull(ReferenceConfigCache.getCache().get(referenceConfig));
+        }
 
+        ReferenceBean referenceBean = referenceBeanManager.get("helloService");
         if ("helloService".equals(referenceBean.getId())) {
-            Assert.assertNotNull(referenceBean.getMethods());
+            ReferenceConfig referenceConfig = referenceBean.getReferenceConfig();
+            Assert.assertNotNull(referenceConfig.getMethods());
         }
     }
 
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilderTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilderTest.java
index 23e754f..b95734f 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilderTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/beans/factory/annotation/ReferenceBeanBuilderTest.java
@@ -17,15 +17,14 @@
 package org.apache.dubbo.config.spring.beans.factory.annotation;
 
 
+import org.apache.dubbo.config.ReferenceConfig;
 import org.apache.dubbo.config.annotation.DubboReference;
 import org.apache.dubbo.config.annotation.Reference;
-import org.apache.dubbo.config.spring.ReferenceBean;
-import org.apache.dubbo.rpc.model.ApplicationModel;
-
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.junit.Assert;
-import org.junit.Before;
 import org.junit.Test;
 import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.runner.RunWith;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.context.ApplicationContext;
@@ -81,9 +80,9 @@ public class ReferenceBeanBuilderTest {
     @Autowired
     private ApplicationContext context;
 
-    @Before
+    @BeforeEach
     public void init() {
-        ApplicationModel.reset();
+        DubboBootstrap.reset();
     }
 
     @Test
@@ -91,8 +90,8 @@ public class ReferenceBeanBuilderTest {
         DubboReference reference = findAnnotation(findField(getClass(), "TEST_FIELD"), DubboReference.class);
         AnnotationAttributes attributes = AnnotationUtils.getAnnotationAttributes(reference, false, false);
         ReferenceBeanBuilder beanBuilder = ReferenceBeanBuilder.create(attributes, context);
-        beanBuilder.interfaceClass(CharSequence.class);
-        ReferenceBean referenceBean = beanBuilder.build();
+        beanBuilder.defaultInterfaceClass(CharSequence.class);
+        ReferenceConfig referenceBean = beanBuilder.build();
         Assert.assertEquals(CharSequence.class, referenceBean.getInterfaceClass());
         Assert.assertEquals("1.0.0", referenceBean.getVersion());
         Assert.assertEquals("TEST_GROUP", referenceBean.getGroup());
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboComponentScanRegistrarTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboComponentScanRegistrarTest.java
index 80a8d40..2b89f0f 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboComponentScanRegistrarTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboComponentScanRegistrarTest.java
@@ -16,12 +16,11 @@
  */
 package org.apache.dubbo.config.spring.context.annotation;
 
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.apache.dubbo.config.spring.api.DemoService;
 import org.apache.dubbo.config.spring.context.annotation.consumer.ConsumerConfiguration;
 import org.apache.dubbo.config.spring.context.annotation.provider.DemoServiceImpl;
 import org.apache.dubbo.config.spring.context.annotation.provider.ProviderConfiguration;
-import org.apache.dubbo.rpc.model.ApplicationModel;
-
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
@@ -41,12 +40,11 @@ public class DubboComponentScanRegistrarTest {
 
     @BeforeEach
     public void setUp() {
-        ApplicationModel.reset();
+        DubboBootstrap.reset();
     }
 
     @AfterEach
     public void tearDown() {
-        ApplicationModel.reset();
     }
 
     @Test
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboConfigConfigurationTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboConfigConfigurationTest.java
index 724f132..c2696fc 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboConfigConfigurationTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/DubboConfigConfigurationTest.java
@@ -21,6 +21,7 @@ import org.apache.dubbo.config.ModuleConfig;
 import org.apache.dubbo.config.ProtocolConfig;
 import org.apache.dubbo.config.RegistryConfig;
 
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
@@ -41,6 +42,7 @@ public class DubboConfigConfigurationTest {
 
     @BeforeEach
     public void before() throws IOException {
+        DubboBootstrap.reset();
 
         context = new AnnotationConfigApplicationContext();
         ResourcePropertySource propertySource = new ResourcePropertySource("META-INF/config.properties");
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboConfigTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboConfigTest.java
index 5817185..6ecaffb 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboConfigTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboConfigTest.java
@@ -24,8 +24,11 @@ import org.apache.dubbo.config.ProtocolConfig;
 import org.apache.dubbo.config.ProviderConfig;
 import org.apache.dubbo.config.RegistryConfig;
 
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.junit.Assert;
+import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.jupiter.api.Test;
 import org.springframework.context.annotation.AnnotationConfigApplicationContext;
 import org.springframework.context.annotation.PropertySource;
@@ -44,6 +47,15 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
  */
 public class EnableDubboConfigTest {
 
+    @BeforeEach
+    public void setUp() {
+        DubboBootstrap.reset();
+    }
+
+    @AfterEach
+    public void tearDown() {
+    }
+
     @Test
     public void testSingle() {
 
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboTest.java
index d9364d0..3a56d41 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/context/annotation/EnableDubboTest.java
@@ -17,11 +17,10 @@
 package org.apache.dubbo.config.spring.context.annotation;
 
 import org.apache.dubbo.config.RegistryConfig;
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.apache.dubbo.config.spring.api.DemoService;
 import org.apache.dubbo.config.spring.context.annotation.consumer.test.TestConsumerConfiguration;
 import org.apache.dubbo.config.spring.context.annotation.provider.DemoServiceImpl;
-import org.apache.dubbo.rpc.model.ApplicationModel;
-
 import org.junit.jupiter.api.AfterEach;
 import org.junit.jupiter.api.Assertions;
 import org.junit.jupiter.api.BeforeEach;
@@ -52,14 +51,13 @@ public class EnableDubboTest {
 
     @BeforeEach
     public void setUp() {
-        ApplicationModel.reset();
         context = new AnnotationConfigApplicationContext();
+        DubboBootstrap.reset();
     }
 
     @AfterEach
     public void tearDown() {
-        ApplicationModel.reset();
-        context.close();
+        //context.close();
     }
 
     @Test
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/issues/Issue6252Test.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/issues/Issue6252Test.java
index 8c34e7e..d09407a 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/issues/Issue6252Test.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/issues/Issue6252Test.java
@@ -16,9 +16,13 @@
  */
 package org.apache.dubbo.config.spring.issues;
 
+import org.apache.dubbo.config.annotation.DubboReference;
+import org.apache.dubbo.config.annotation.DubboService;
 import org.apache.dubbo.config.spring.ReferenceBean;
+import org.apache.dubbo.config.spring.api.DemoService;
 import org.apache.dubbo.config.spring.context.annotation.EnableDubboConfig;
 
+import org.apache.dubbo.config.spring.impl.DemoServiceImpl;
 import org.junit.jupiter.api.Test;
 import org.springframework.context.annotation.AnnotationConfigApplicationContext;
 import org.springframework.context.annotation.Bean;
@@ -35,15 +39,13 @@ import org.springframework.context.annotation.PropertySource;
 @PropertySource("classpath:/META-INF/issue-6252-test.properties")
 public class Issue6252Test {
 
-    @Bean
-    public static ReferenceBean referenceBean() {
-        return new ReferenceBean();
-    }
+    @DubboReference
+    private DemoService demoService;
 
     @Test
     public void test() throws Exception {
         AnnotationConfigApplicationContext context = new AnnotationConfigApplicationContext(Issue6252Test.class);
-        context.getBean(ReferenceBean.class);
+        DemoService demoService = context.getBean(DemoService.class);
         context.close();
     }
 
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/DubboNamespaceHandlerTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/DubboNamespaceHandlerTest.java
index 705b890..31c16f5 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/DubboNamespaceHandlerTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/DubboNamespaceHandlerTest.java
@@ -21,6 +21,7 @@ import org.apache.dubbo.config.ModuleConfig;
 import org.apache.dubbo.config.MonitorConfig;
 import org.apache.dubbo.config.ProtocolConfig;
 import org.apache.dubbo.config.ProviderConfig;
+import org.apache.dubbo.config.RegistryConfig;
 import org.apache.dubbo.config.spring.ConfigTest;
 import org.apache.dubbo.config.spring.ServiceBean;
 import org.apache.dubbo.config.spring.api.DemoService;
@@ -47,6 +48,9 @@ import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
 
 public class DubboNamespaceHandlerTest {
+
+    private static String resourcePath = ConfigTest.class.getPackage().getName().replace('.', '/');
+
     @BeforeEach
     public void setUp() {
         ApplicationModel.reset();
@@ -75,8 +79,8 @@ public class DubboNamespaceHandlerTest {
     @Test
     public void testProviderXml() {
         ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(
-                ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider.xml",
-                ConfigTest.class.getPackage().getName().replace('.', '/') + "/demo-provider-properties.xml"
+                resourcePath + "/demo-provider.xml",
+                resourcePath + "/demo-provider-properties.xml"
         );
         ctx.start();
 
@@ -93,13 +97,17 @@ public class DubboNamespaceHandlerTest {
         assertThat(applicationConfig, not(nullValue()));
         assertThat(applicationConfig.getName(), is("demo-provider"));
 
+        RegistryConfig registryConfig = context.getBean(RegistryConfig.class);
+        assertThat(registryConfig, not(nullValue()));
+        assertThat(registryConfig.getAddress(), is("N/A"));
+
         DemoService service = context.getBean(DemoService.class);
         assertThat(service, not(nullValue()));
     }
 
     @Test
     public void testMultiProtocol() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-protocol.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-protocol.xml");
         ctx.start();
 
         Map<String, ProtocolConfig> protocolConfigMap = ctx.getBeansOfType(ProtocolConfig.class);
@@ -114,7 +122,7 @@ public class DubboNamespaceHandlerTest {
 
     @Test
     public void testDefaultProtocol() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/override-protocol.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/override-protocol.xml");
         ctx.start();
 
         ProtocolConfig protocolConfig = ctx.getBean(ProtocolConfig.class);
@@ -124,7 +132,7 @@ public class DubboNamespaceHandlerTest {
 
     @Test
     public void testCustomParameter() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/customize-parameter.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/customize-parameter.xml");
         ctx.start();
 
         ProtocolConfig protocolConfig = ctx.getBean(ProtocolConfig.class);
@@ -139,7 +147,7 @@ public class DubboNamespaceHandlerTest {
 
     @Test
     public void testDelayFixedTime() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext("classpath:/" + ConfigTest.class.getPackage().getName().replace('.', '/') + "/delay-fixed-time.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext("classpath:/" + resourcePath + "/delay-fixed-time.xml");
         ctx.start();
 
         assertThat(ctx.getBean(ServiceBean.class).getDelay(), is(300));
@@ -147,7 +155,7 @@ public class DubboNamespaceHandlerTest {
 
     @Test
     public void testTimeoutConfig() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/provider-nested-service.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/provider-nested-service.xml");
         ctx.start();
 
         Map<String, ProviderConfig> providerConfigMap = ctx.getBeansOfType(ProviderConfig.class);
@@ -157,7 +165,7 @@ public class DubboNamespaceHandlerTest {
 
     @Test
     public void testMonitor() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/provider-with-monitor.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/provider-with-monitor.xml");
         ctx.start();
 
         assertThat(ctx.getBean(MonitorConfig.class), not(nullValue()));
@@ -166,7 +174,7 @@ public class DubboNamespaceHandlerTest {
 //    @Test
 //    public void testMultiMonitor() {
 //        Assertions.assertThrows(BeanCreationException.class, () -> {
-//            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/multi-monitor.xml");
+//            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/multi-monitor.xml");
 //            ctx.start();
 //        });
 //    }
@@ -174,14 +182,14 @@ public class DubboNamespaceHandlerTest {
 //    @Test
 //    public void testMultiProviderConfig() {
 //        Assertions.assertThrows(BeanCreationException.class, () -> {
-//            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/provider-multi.xml");
+//            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/provider-multi.xml");
 //            ctx.start();
 //        });
 //    }
 
     @Test
     public void testModuleInfo() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/provider-with-module.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/provider-with-module.xml");
         ctx.start();
 
         ModuleConfig moduleConfig = ctx.getBean(ModuleConfig.class);
@@ -191,14 +199,14 @@ public class DubboNamespaceHandlerTest {
     @Test
     public void testNotificationWithWrongBean() {
         Assertions.assertThrows(BeanCreationException.class, () -> {
-            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/consumer-notification.xml");
+            ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/consumer-notification.xml");
             ctx.start();
         });
     }
 
     @Test
     public void testProperty() {
-        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(ConfigTest.class.getPackage().getName().replace('.', '/') + "/service-class.xml");
+        ClassPathXmlApplicationContext ctx = new ClassPathXmlApplicationContext(resourcePath + "/service-class.xml");
         ctx.start();
 
         ServiceBean serviceBean = ctx.getBean(ServiceBean.class);
diff --git a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/GenericServiceTest.java b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/GenericServiceTest.java
index 6deb2ab..e89ff6c 100644
--- a/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/GenericServiceTest.java
+++ b/dubbo-config/dubbo-config-spring/src/test/java/org/apache/dubbo/config/spring/schema/GenericServiceTest.java
@@ -16,13 +16,12 @@
  */
 package org.apache.dubbo.config.spring.schema;
 
+import org.apache.dubbo.config.bootstrap.DubboBootstrap;
 import org.apache.dubbo.config.spring.ReferenceBean;
 import org.apache.dubbo.config.spring.ServiceBean;
-import org.apache.dubbo.rpc.model.ApplicationModel;
-
 import org.junit.After;
-import org.junit.Before;
 import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
 import org.junit.runner.RunWith;
 import org.springframework.beans.factory.annotation.Autowired;
 import org.springframework.beans.factory.annotation.Qualifier;
@@ -37,14 +36,13 @@ import static org.junit.Assert.assertNotNull;
 @ImportResource(locations = "classpath:/META-INF/spring/dubbo-generic-consumer.xml")
 public class GenericServiceTest {
 
-    @Before
+    @BeforeEach
     public void setUp() {
-        ApplicationModel.reset();
+        DubboBootstrap.reset();
     }
 
     @After
     public void tearDown() {
-        ApplicationModel.reset();
     }
 
     @Autowired
diff --git a/dubbo-config/dubbo-config-spring/src/test/resources/META-INF/init-reference.properties b/dubbo-config/dubbo-config-spring/src/test/resources/META-INF/init-reference.properties
new file mode 100644
index 0000000..c6f6d6a
--- /dev/null
+++ b/dubbo-config/dubbo-config-spring/src/test/resources/META-INF/init-reference.properties
@@ -0,0 +1,5 @@
+# The properties for org/apache/dubbo/config/spring/init-reference*.xml
+
+call.timeout=100
+connection.timeout=1000
+sayName.retry=false
\ No newline at end of file
diff --git a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/demo-provider.xml b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/demo-provider.xml
index d5d3c52..dc77848 100644
--- a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/demo-provider.xml
+++ b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/demo-provider.xml
@@ -21,16 +21,16 @@
     http://dubbo.apache.org/schema/dubbo http://dubbo.apache.org/schema/dubbo/dubbo.xsd">
 
     <!-- current application configuration -->
-    <dubbo:application name="${dubbo.application.name}"/>
+    <dubbo:application name="${dubbo.application.name:foo-app}"/>
 
     <!-- registry center configuration -->
-    <dubbo:registry address="${dubbo.registry.address}"/>
+    <dubbo:registry address="${dubbo.registry.address:foo-address}"/>
 
     <!-- service protocol configuration -->
-    <dubbo:protocol name="${dubbo.protocol.name}" port="${dubbo.protocol.port}"/>
+    <dubbo:protocol name="${dubbo.protocol.name:foo-protocol}" port="${dubbo.protocol.port:20881}"/>
 
     <!-- service configuration -->
-    <dubbo:service interface="org.apache.dubbo.config.spring.api.DemoService" ref="demoService"/>
+    <dubbo:service interface="org.apache.dubbo.config.spring.api.DemoService" group="demo" version="1.2.3" ref="demoService"/>
 
     <bean id="demoService" class="org.apache.dubbo.config.spring.impl.DemoServiceImpl"/>
 
diff --git a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference-properties.xml
similarity index 68%
copy from dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml
copy to dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference-properties.xml
index 360922f..08cb609 100644
--- a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml
+++ b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference-properties.xml
@@ -15,17 +15,11 @@
   limitations under the License.
   -->
 <beans xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
-       xmlns:dubbo="http://dubbo.apache.org/schema/dubbo"
        xmlns="http://www.springframework.org/schema/beans"
-       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-4.3.xsd
-    http://dubbo.apache.org/schema/dubbo http://dubbo.apache.org/schema/dubbo/dubbo.xsd
-    ">
+       xsi:schemaLocation="http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-4.3.xsd">
 
-    <!-- current application configuration -->
-    <dubbo:application name="demo-consumer"/>
-
-    <!-- service configuration -->
-    <dubbo:reference id="demoService" interface="org.apache.dubbo.config.spring.api.DemoService"
-                     url="dubbo://127.0.0.1:20813" init="true" scope="remote"/>
+    <bean class="org.springframework.context.support.PropertySourcesPlaceholderConfigurer">
+        <property name="location" value="classpath:/META-INF/init-reference.properties"/>
+    </bean>
 
 </beans>
\ No newline at end of file
diff --git a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml
index 360922f..10e9f84 100644
--- a/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml
+++ b/dubbo-config/dubbo-config-spring/src/test/resources/org/apache/dubbo/config/spring/init-reference.xml
@@ -24,8 +24,22 @@
     <!-- current application configuration -->
     <dubbo:application name="demo-consumer"/>
 
-    <!-- service configuration -->
-    <dubbo:reference id="demoService" interface="org.apache.dubbo.config.spring.api.DemoService"
-                     url="dubbo://127.0.0.1:20813" init="true" scope="remote"/>
+    <dubbo:registry id="my-registry" address="zookeeper://localhost:2181" />
 
+    <dubbo:consumer id="my-consumer" registry="my-registry" group="demo" version="1.2.3" scope="remote" init="true"
+                    timeout="${call.timeout:foo100}" >
+        <dubbo:reference id="demoService2" interface="org.apache.dubbo.config.spring.api.DemoService" generic="true"
+                         url="dubbo://127.0.0.1:20813" />
+    </dubbo:consumer>
+
+    <!-- service reference configuration -->
+    <dubbo:reference id="demoService" interface="org.apache.dubbo.config.spring.api.DemoService" group="demo" version="1.2.3"
+                     url="dubbo://127.0.0.1:20813" init="true" timeout="${call.timeout:foo100}"
+                     scope="remote" protocol="dubbo" registry="my-registry" consumer="my-consumer" >
+        <dubbo:parameter key="connec.timeout" value="${connection.timeout:foo1000}"/>
+        <dubbo:method name="sayName" retry="${sayName.retry:foo-retry}">
+            <dubbo:argument index="0" callback="true" />
+            <dubbo:parameter key="access-token" value="my-token" />
+        </dubbo:method>
+    </dubbo:reference>
 </beans>
\ No newline at end of file
diff --git a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/definition/util/ClassUtils.java b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/definition/util/ClassUtils.java
index ffa7e2c..04096da 100755
--- a/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/definition/util/ClassUtils.java
+++ b/dubbo-metadata/dubbo-metadata-api/src/main/java/org/apache/dubbo/metadata/definition/util/ClassUtils.java
@@ -16,8 +16,6 @@
  */
 package org.apache.dubbo.metadata.definition.util;
 
-import sun.reflect.generics.reflectiveObjects.ParameterizedTypeImpl;
-
 import java.lang.reflect.Field;
 import java.lang.reflect.Method;
 import java.lang.reflect.Modifier;
@@ -125,7 +123,7 @@ public final class ClassUtils {
 
             sb.append(".");
 
-            if (ownerType instanceof ParameterizedTypeImpl) {
+            if (ownerType instanceof ParameterizedType) {
                 // Find simple name of nested type by removing the
                 // shared prefix with owner.
                 sb.append(rawType.getName().replace(((Class) ((ParameterizedType) ownerType).getRawType()).getName() + "$",
diff --git a/dubbo-registry/dubbo-registry-multicast/src/test/java/org/apache/dubbo/registry/multicast/MulticastRegistryTest.java b/dubbo-registry/dubbo-registry-multicast/src/test/java/org/apache/dubbo/registry/multicast/MulticastRegistryTest.java
index f0b9f1e..598f34a 100644
--- a/dubbo-registry/dubbo-registry-multicast/src/test/java/org/apache/dubbo/registry/multicast/MulticastRegistryTest.java
+++ b/dubbo-registry/dubbo-registry-multicast/src/test/java/org/apache/dubbo/registry/multicast/MulticastRegistryTest.java
@@ -26,6 +26,7 @@ import org.junit.jupiter.api.Test;
 
 import java.net.InetAddress;
 import java.net.MulticastSocket;
+import java.net.UnknownHostException;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -57,9 +58,13 @@ public class MulticastRegistryTest {
      */
     @Test
     public void testUrlError() {
-        Assertions.assertThrows(IllegalStateException.class, () -> {
-            URL errorUrl = URL.valueOf("multicast://mullticast/");
-            new MulticastRegistry(errorUrl);
+        Assertions.assertThrows(UnknownHostException.class, () -> {
+            try {
+                URL errorUrl = URL.valueOf("multicast://mullticast.local/");
+                new MulticastRegistry(errorUrl);
+            } catch (IllegalStateException e) {
+                throw e.getCause();
+            }
         });
     }
 
diff --git a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/proxy/AbstractProxyFactory.java b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/proxy/AbstractProxyFactory.java
index 837750f..ab1ab44 100644
--- a/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/proxy/AbstractProxyFactory.java
+++ b/dubbo-rpc/dubbo-rpc-api/src/main/java/org/apache/dubbo/rpc/proxy/AbstractProxyFactory.java
@@ -79,6 +79,10 @@ public abstract class AbstractProxyFactory implements ProxyFactory {
         return getProxy(invoker, interfaces.toArray(new Class<?>[0]));
     }
 
+    public static Class<?>[] getInternalInterfaces() {
+        return INTERNAL_INTERFACES.clone();
+    }
+
     public abstract <T> T getProxy(Invoker<T> invoker, Class<?>[] types);
 
 }