diff --git a/README.md b/README.md
index adb91e54e119ccec34a735315c3c38505f41e3b9..251221c693b52c47d5663b0999fbf948008eafb2 100644
--- a/README.md
+++ b/README.md
@@ -1,6 +1,6 @@
 # dubbo-go #
 ---
-Apache Dubbo golang version.
+Apache Dubbo Golang Implementation.
 
 ## License
 
@@ -20,6 +20,7 @@ Apache License, Version 2.0
 
 The subdirectory examples shows how to use dubbo-go. Please read the examples/readme.md carefully to learn how to dispose the configuration and compile the program.
 
+
 ## Todo list
 
 - [ ] Tcp Transport and Hessian2 protocol
diff --git a/client/client_transport.go b/client/client_transport.go
deleted file mode 100644
index a60faab284aca03e486d1a3b934baaa9e4a8f1e5..0000000000000000000000000000000000000000
--- a/client/client_transport.go
+++ /dev/null
@@ -1,22 +0,0 @@
-package client
-
-import (
-	"context"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-type Transport interface {
-	Call(ctx context.Context, url registry.ServiceURL, request Request, resp interface{}) error
-	NewRequest(conf registry.ServiceConfig, method string, args interface{}) (Request, error)
-}
-
-//////////////////////////////////////////////
-// Request
-//////////////////////////////////////////////
-
-type Request interface {
-	ServiceConfig() registry.ServiceConfig
-}
diff --git a/client/invoker/invoker.go b/client/invoker/invoker.go
deleted file mode 100644
index 1cc205d218fd69c13f8e91f3266a6e68bb6df114..0000000000000000000000000000000000000000
--- a/client/invoker/invoker.go
+++ /dev/null
@@ -1,224 +0,0 @@
-package invoker
-
-import (
-	"context"
-	"sync"
-	"time"
-)
-
-import (
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/client/selector"
-	"github.com/dubbo/dubbo-go/dubbo"
-	"github.com/dubbo/dubbo-go/jsonrpc"
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-const RegistryConnDelay = 3
-
-type Options struct {
-	ServiceTTL time.Duration
-	selector   selector.Selector
-	//TODO:we should provider a transport client interface
-	HttpClient  *jsonrpc.HTTPClient
-	DubboClient *dubbo.Client
-}
-type Option func(*Options)
-
-func WithServiceTTL(ttl time.Duration) Option {
-	return func(o *Options) {
-		o.ServiceTTL = ttl
-	}
-}
-
-func WithHttpClient(client *jsonrpc.HTTPClient) Option {
-	return func(o *Options) {
-		o.HttpClient = client
-	}
-}
-func WithDubboClient(client *dubbo.Client) Option {
-	return func(o *Options) {
-		o.DubboClient = client
-	}
-}
-
-func WithLBSelector(selector selector.Selector) Option {
-	return func(o *Options) {
-		o.selector = selector
-	}
-}
-
-type Invoker struct {
-	Options
-	cacheServiceMap map[string]*ServiceArray
-	registry        registry.Registry
-	listenerLock    sync.Mutex
-}
-
-func NewInvoker(registry registry.Registry, opts ...Option) (*Invoker, error) {
-	options := Options{
-		//default 300s
-		ServiceTTL: time.Duration(300e9),
-		selector:   selector.NewRandomSelector(),
-	}
-	for _, opt := range opts {
-		opt(&options)
-	}
-	if options.HttpClient == nil && options.DubboClient == nil {
-		return nil, jerrors.New("Must specify the transport client!")
-	}
-	invoker := &Invoker{
-		Options:         options,
-		cacheServiceMap: make(map[string]*ServiceArray),
-		registry:        registry,
-	}
-	go invoker.listen()
-	return invoker, nil
-}
-
-func (ivk *Invoker) listen() {
-	for {
-		if ivk.registry.IsClosed() {
-			log.Warn("event listener game over.")
-			return
-		}
-
-		listener, err := ivk.registry.Subscribe()
-		if err != nil {
-			if ivk.registry.IsClosed() {
-				log.Warn("event listener game over.")
-				return
-			}
-			log.Warn("getListener() = err:%s", jerrors.ErrorStack(err))
-			time.Sleep(time.Duration(RegistryConnDelay) * time.Second)
-			continue
-		}
-
-		for {
-			if serviceEvent, err := listener.Next(); err != nil {
-				log.Warn("Selector.watch() = error{%v}", jerrors.ErrorStack(err))
-				listener.Close()
-				time.Sleep(time.Duration(RegistryConnDelay) * time.Second)
-				return
-			} else {
-				ivk.update(serviceEvent)
-			}
-
-		}
-
-	}
-}
-
-func (ivk *Invoker) update(res *registry.ServiceEvent) {
-	if res == nil || res.Service == nil {
-		return
-	}
-
-	log.Debug("registry update, result{%s}", res)
-	registryKey := res.Service.ServiceConfig().Key()
-
-	ivk.listenerLock.Lock()
-	defer ivk.listenerLock.Unlock()
-
-	svcArr, ok := ivk.cacheServiceMap[registryKey]
-	log.Debug("registry name:%s, its current member lists:%+v", registryKey, svcArr)
-
-	switch res.Action {
-	case registry.ServiceAdd:
-		if ok {
-			svcArr.add(res.Service, ivk.ServiceTTL)
-		} else {
-			ivk.cacheServiceMap[registryKey] = newServiceArray([]registry.ServiceURL{res.Service})
-		}
-	case registry.ServiceDel:
-		if ok {
-			svcArr.del(res.Service, ivk.ServiceTTL)
-			if len(svcArr.arr) == 0 {
-				delete(ivk.cacheServiceMap, registryKey)
-				log.Warn("delete registry %s from registry map", registryKey)
-			}
-		}
-		log.Error("selector delete registryURL{%s}", res.Service)
-	}
-}
-
-func (ivk *Invoker) getService(registryConf registry.ServiceConfig) (*ServiceArray, error) {
-	defer ivk.listenerLock.Unlock()
-
-	registryKey := registryConf.Key()
-
-	ivk.listenerLock.Lock()
-	svcArr, sok := ivk.cacheServiceMap[registryKey]
-	log.Debug("r.svcArr[registryString{%v}] = svcArr{%s}", registryKey, svcArr)
-	if sok && time.Since(svcArr.birth) < ivk.Options.ServiceTTL {
-		return svcArr, nil
-	}
-	ivk.listenerLock.Unlock()
-
-	svcs, err := ivk.registry.GetService(registryConf)
-	ivk.listenerLock.Lock()
-
-	if err != nil {
-		log.Error("Registry.get(conf:%+v) = {err:%s, svcs:%+v}",
-			registryConf, jerrors.ErrorStack(err), svcs)
-
-		return nil, jerrors.Trace(err)
-	}
-
-	newSvcArr := newServiceArray(svcs)
-	ivk.cacheServiceMap[registryKey] = newSvcArr
-	return newSvcArr, nil
-}
-
-func (ivk *Invoker) HttpCall(ctx context.Context, reqId int64, req client.Request, resp interface{}) error {
-
-	serviceConf := req.ServiceConfig()
-	registryArray, err := ivk.getService(serviceConf)
-	if err != nil {
-		return err
-	}
-	if len(registryArray.arr) == 0 {
-		return jerrors.New("cannot find svc " + serviceConf.String())
-	}
-	url, err := ivk.selector.Select(reqId, registryArray)
-	if err != nil {
-		return err
-	}
-	if err = ivk.HttpClient.Call(ctx, url, req, resp); err != nil {
-		log.Error("client.Call() return error:%+v", jerrors.ErrorStack(err))
-		return err
-	}
-	log.Info("response result:%s", resp)
-	return nil
-}
-
-func (ivk *Invoker) DubboCall(reqId int64, registryConf registry.ServiceConfig, method string, args, reply interface{}, opts ...dubbo.CallOption) error {
-
-	registryArray, err := ivk.getService(registryConf)
-	if err != nil {
-		return err
-	}
-	if len(registryArray.arr) == 0 {
-		return jerrors.New("cannot find svc " + registryConf.String())
-	}
-	url, err := ivk.selector.Select(reqId, registryArray)
-	if err != nil {
-		return err
-	}
-	//TODO:这里要改一下call方法改为接收指针类型
-	if err = ivk.DubboClient.Call(url.Ip()+":"+url.Port(), url, method, args, reply, opts...); err != nil {
-		log.Error("client.Call() return error:%+v", jerrors.ErrorStack(err))
-		return err
-	}
-	log.Info("response result:%s", reply)
-	return nil
-}
-
-func (ivk *Invoker) Close() {
-	ivk.DubboClient.Close()
-}
diff --git a/client/selector/random.go b/client/selector/random.go
deleted file mode 100644
index 02f27bafdc55d48d6f63cee26e745b8759e98589..0000000000000000000000000000000000000000
--- a/client/selector/random.go
+++ /dev/null
@@ -1,27 +0,0 @@
-package selector
-
-import (
-	"math/rand"
-	"sync/atomic"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-type RandomSelector struct{}
-
-func NewRandomSelector() Selector {
-	return &RandomSelector{}
-}
-
-func (s *RandomSelector) Select(ID int64, array client.ServiceArrayIf) (registry.ServiceURL, error) {
-	if array.GetSize() == 0 {
-		return nil, ServiceArrayEmpty
-	}
-
-	idx := atomic.AddInt64(array.GetIdx(), 1)
-	idx = ((int64)(rand.Int()) + ID) % array.GetSize()
-	return array.GetService(idx), nil
-}
diff --git a/client/selector/round_robin.go b/client/selector/round_robin.go
deleted file mode 100644
index 3a429c63ce9e36d3487f8e388382b269ba1cb83a..0000000000000000000000000000000000000000
--- a/client/selector/round_robin.go
+++ /dev/null
@@ -1,26 +0,0 @@
-package selector
-
-import (
-	"sync/atomic"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-type RoundRobinSelector struct{}
-
-func NewRoundRobinSelector() Selector {
-	return &RoundRobinSelector{}
-}
-
-func (s *RoundRobinSelector) Select(ID int64, array client.ServiceArrayIf) (registry.ServiceURL, error) {
-	if array.GetSize() == 0 {
-		return nil, ServiceArrayEmpty
-	}
-
-	idx := atomic.AddInt64(array.GetIdx(), 1)
-	idx = (ID + idx) % array.GetSize()
-	return array.GetService(idx), nil
-}
diff --git a/client/selector/selector.go b/client/selector/selector.go
deleted file mode 100644
index 7f077c48b143b05ec71b66ae12a7f2cc1d24e6d0..0000000000000000000000000000000000000000
--- a/client/selector/selector.go
+++ /dev/null
@@ -1,18 +0,0 @@
-package selector
-
-import (
-	"fmt"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-var (
-	ServiceArrayEmpty = fmt.Errorf("emtpy service array")
-)
-
-type Selector interface {
-	Select(ID int64, array client.ServiceArrayIf) (registry.ServiceURL, error)
-}
diff --git a/client/service_array.go b/client/service_array.go
deleted file mode 100644
index af1aa9c1474cab3178137611274dabd7d6869626..0000000000000000000000000000000000000000
--- a/client/service_array.go
+++ /dev/null
@@ -1,9 +0,0 @@
-package client
-
-import "github.com/dubbo/dubbo-go/registry"
-
-type ServiceArrayIf interface {
-	GetIdx() *int64
-	GetSize() int64
-	GetService(i int64) registry.ServiceURL
-}
diff --git a/cluster/cluster.go b/cluster/cluster.go
new file mode 100644
index 0000000000000000000000000000000000000000..c39b8a28b21146d5497b123e4969cda7d7bb3dcf
--- /dev/null
+++ b/cluster/cluster.go
@@ -0,0 +1,7 @@
+package cluster
+
+import "github.com/dubbo/go-for-apache-dubbo/protocol"
+
+type Cluster interface {
+	Join(Directory) protocol.Invoker
+}
diff --git a/cluster/directory.go b/cluster/directory.go
new file mode 100644
index 0000000000000000000000000000000000000000..9146c593e60b2e63c2bae5947fab495d2ebb03a2
--- /dev/null
+++ b/cluster/directory.go
@@ -0,0 +1,12 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+// Extension - Directory
+type Directory interface {
+	common.Node
+	List(invocation protocol.Invocation) []protocol.Invoker
+}
diff --git a/cluster/directory/base_directory.go b/cluster/directory/base_directory.go
new file mode 100644
index 0000000000000000000000000000000000000000..7b38a63f2c62e37ce1f2242707b3f1f00709b74f
--- /dev/null
+++ b/cluster/directory/base_directory.go
@@ -0,0 +1,32 @@
+package directory
+
+import (
+	"github.com/tevino/abool"
+)
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
+
+type BaseDirectory struct {
+	url       *config.URL
+	destroyed *abool.AtomicBool
+}
+
+func NewBaseDirectory(url *config.URL) BaseDirectory {
+	return BaseDirectory{
+		url:       url,
+		destroyed: abool.NewBool(false),
+	}
+}
+func (dir *BaseDirectory) GetUrl() config.URL {
+	return *dir.url
+}
+
+func (dir *BaseDirectory) Destroy() {
+	if dir.destroyed.SetToIf(false, true) {
+	}
+}
+
+func (dir *BaseDirectory) IsAvailable() bool {
+	return !dir.destroyed.IsSet()
+}
diff --git a/client/invoker/service_array.go b/cluster/directory/service_array.go
similarity index 66%
rename from client/invoker/service_array.go
rename to cluster/directory/service_array.go
index 1f78d722ab0e4d948095857f7cb33736f8d956d4..d7d8821e4e1ba492c5f090e1b2bcae9a42e7f3b5 100644
--- a/client/invoker/service_array.go
+++ b/cluster/directory/service_array.go
@@ -1,6 +1,7 @@
-package invoker
+package directory
 
 import (
+	"context"
 	"fmt"
 	"strings"
 	"time"
@@ -10,9 +11,7 @@ import (
 	jerrors "github.com/juju/errors"
 )
 
-import (
-	"github.com/dubbo/dubbo-go/registry"
-)
+import "github.com/dubbo/go-for-apache-dubbo/config"
 
 //////////////////////////////////////////
 // registry array
@@ -25,15 +24,17 @@ var (
 )
 
 type ServiceArray struct {
-	arr   []registry.ServiceURL
-	birth time.Time
-	idx   int64
+	context context.Context
+	arr     []config.URL
+	birth   time.Time
+	idx     int64
 }
 
-func newServiceArray(arr []registry.ServiceURL) *ServiceArray {
+func NewServiceArray(ctx context.Context, arr []config.URL) *ServiceArray {
 	return &ServiceArray{
-		arr:   arr,
-		birth: time.Now(),
+		context: ctx,
+		arr:     arr,
+		birth:   time.Now(),
 	}
 }
 
@@ -45,7 +46,7 @@ func (s *ServiceArray) GetSize() int64 {
 	return int64(len(s.arr))
 }
 
-func (s *ServiceArray) GetService(i int64) registry.ServiceURL {
+func (s *ServiceArray) GetService(i int64) config.URL {
 	return s.arr[i]
 }
 
@@ -60,14 +61,14 @@ func (s *ServiceArray) String() string {
 	return builder.String()
 }
 
-func (s *ServiceArray) add(registry registry.ServiceURL, ttl time.Duration) {
-	s.arr = append(s.arr, registry)
+func (s *ServiceArray) Add(url config.URL, ttl time.Duration) {
+	s.arr = append(s.arr, url)
 	s.birth = time.Now().Add(ttl)
 }
 
-func (s *ServiceArray) del(registry registry.ServiceURL, ttl time.Duration) {
+func (s *ServiceArray) Del(url config.URL, ttl time.Duration) {
 	for i, svc := range s.arr {
-		if svc.PrimitiveURL() == registry.PrimitiveURL() {
+		if svc.PrimitiveURL == url.PrimitiveURL {
 			s.arr = append(s.arr[:i], s.arr[i+1:]...)
 			s.birth = time.Now().Add(ttl)
 			break
diff --git a/cluster/directory/static_directory.go b/cluster/directory/static_directory.go
new file mode 100644
index 0000000000000000000000000000000000000000..80ff645368ca5a2d6fb27c50d0ce59301e51cff4
--- /dev/null
+++ b/cluster/directory/static_directory.go
@@ -0,0 +1,32 @@
+package directory
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type StaticDirectory struct {
+	BaseDirectory
+	invokers []protocol.Invoker
+}
+
+func NewStaticDirectory(invokers []protocol.Invoker) *StaticDirectory {
+	return &StaticDirectory{
+		BaseDirectory: NewBaseDirectory(nil),
+		invokers:      invokers,
+	}
+}
+
+//for-loop invokers ,if all invokers is available ,then it means directory is available
+func (dir *StaticDirectory) IsAvailable() bool {
+	for _, invoker := range dir.invokers {
+		if !invoker.IsAvailable() {
+			return false
+		}
+	}
+	return true
+}
+
+func (dir *StaticDirectory) List(invocation protocol.Invocation) []protocol.Invoker {
+	//TODO:Here should add router
+	return dir.invokers
+}
diff --git a/cluster/loadbalance.go b/cluster/loadbalance.go
new file mode 100644
index 0000000000000000000000000000000000000000..18f183e937ebd1caeb22b1814ea5190296011e0b
--- /dev/null
+++ b/cluster/loadbalance.go
@@ -0,0 +1,37 @@
+package cluster
+
+import (
+	"time"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+// Extension - LoadBalance
+type LoadBalance interface {
+	Select([]protocol.Invoker, config.URL, protocol.Invocation) protocol.Invoker
+}
+
+func GetWeight(invoker protocol.Invoker, invocation protocol.Invocation) int64 {
+	url := invoker.GetUrl()
+	weight := url.GetMethodParamInt(invocation.MethodName(), constant.WEIGHT_KEY, constant.DEFAULT_WEIGHT)
+	if weight > 0 {
+		//get service register time an do warm up time
+		now := time.Now().Unix()
+		timestamp := url.GetParamInt(constant.REMOTE_TIMESTAMP_KEY, now)
+		if uptime := now - timestamp; uptime > 0 {
+			warmup := url.GetParamInt(constant.WARMUP_KEY, constant.DEFAULT_WARMUP)
+			if uptime < warmup {
+				if ww := float64(uptime) / float64(warmup) / float64(weight); ww < 1 {
+					weight = 1
+				} else if int64(ww) <= weight {
+					weight = int64(ww)
+				}
+			}
+		}
+	}
+	return weight
+}
diff --git a/cluster/loadbalance/random.go b/cluster/loadbalance/random.go
new file mode 100644
index 0000000000000000000000000000000000000000..6251438f9666d7285eb6e6f25a0741390df3c70f
--- /dev/null
+++ b/cluster/loadbalance/random.go
@@ -0,0 +1,62 @@
+package loadbalance
+
+import (
+	"math/rand"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+const name = "random"
+
+func init() {
+	extension.SetLoadbalance(name, NewRandomLoadBalance)
+}
+
+type RandomLoadBalance struct {
+}
+
+func NewRandomLoadBalance() cluster.LoadBalance {
+	return &RandomLoadBalance{}
+}
+
+func (lb *RandomLoadBalance) Select(invokers []protocol.Invoker, url config.URL, invocation protocol.Invocation) protocol.Invoker {
+	var length int
+	if length = len(invokers); length == 1 {
+		return invokers[0]
+	}
+	sameWeight := true
+	weights := make([]int64, length)
+
+	firstWeight := cluster.GetWeight(invokers[0], invocation)
+	totalWeight := firstWeight
+	weights[0] = firstWeight
+
+	for i := 1; i < length; i++ {
+		weight := cluster.GetWeight(invokers[i], invocation)
+		weights[i] = weight
+
+		totalWeight += weight
+		if sameWeight && weight != firstWeight {
+			sameWeight = false
+		}
+	}
+
+	if totalWeight > 0 && !sameWeight {
+		// If (not every invoker has the same weight & at least one invoker's weight>0), select randomly based on totalWeight.
+		offset := rand.Int63n(totalWeight)
+
+		for i := 0; i < length; i++ {
+			offset -= weights[i]
+			if offset < 0 {
+				return invokers[i]
+			}
+		}
+	}
+	// If all invokers have the same weight value or totalWeight=0, return evenly.
+	return invokers[rand.Intn(length)]
+}
diff --git a/cluster/router.go b/cluster/router.go
new file mode 100644
index 0000000000000000000000000000000000000000..828af3a180896d19d98e3af6b1ff70997230789e
--- /dev/null
+++ b/cluster/router.go
@@ -0,0 +1,24 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+// Extension - Router
+
+type RouterFactory interface {
+	Router(config.URL) Router
+}
+
+type Router interface {
+	Route([]protocol.Invoker, config.URL, protocol.Invocation) []protocol.Invoker
+}
+
+type RouterChain struct {
+	routers []Router
+}
+
+func NewRouterChain(url config.URL) {
+
+}
diff --git a/cluster/router/.gitkeep b/cluster/router/.gitkeep
new file mode 100644
index 0000000000000000000000000000000000000000..e69de29bb2d1d6434b8b29ae775ad8c2e48c5391
diff --git a/cluster/support/base_cluster_invoker.go b/cluster/support/base_cluster_invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..1ae288697dc2b4139494d2ae8460eade1b8789d2
--- /dev/null
+++ b/cluster/support/base_cluster_invoker.go
@@ -0,0 +1,109 @@
+package cluster
+
+import (
+	gxnet "github.com/AlexStocks/goext/net"
+	jerrors "github.com/juju/errors"
+	"github.com/tevino/abool"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/version"
+)
+
+type baseClusterInvoker struct {
+	directory      cluster.Directory
+	availablecheck bool
+	destroyed      *abool.AtomicBool
+}
+
+func newBaseClusterInvoker(directory cluster.Directory) baseClusterInvoker {
+	return baseClusterInvoker{
+		directory:      directory,
+		availablecheck: true,
+		destroyed:      abool.NewBool(false),
+	}
+}
+func (invoker *baseClusterInvoker) GetUrl() config.URL {
+	return invoker.directory.GetUrl()
+}
+
+func (invoker *baseClusterInvoker) Destroy() {
+	//this is must atom operation
+	if invoker.destroyed.SetToIf(false, true) {
+		invoker.directory.Destroy()
+	}
+}
+
+func (invoker *baseClusterInvoker) IsAvailable() bool {
+	//TODO:不理解java版本中关于stikyInvoker的逻辑所以先不写
+	return invoker.directory.IsAvailable()
+}
+
+//check invokers availables
+func (invoker *baseClusterInvoker) checkInvokers(invokers []protocol.Invoker, invocation protocol.Invocation) error {
+	if len(invokers) == 0 {
+		ip, _ := gxnet.GetLocalIP()
+		return jerrors.Errorf("Failed to invoke the method %v . No provider available for the service %v from"+
+			"registry %v on the consumer %v using the dubbo version %v .Please check if the providers have been started and registered.",
+			invocation.MethodName(), invoker.directory.GetUrl().Key(), invoker.directory.GetUrl().String(), ip, version.Version)
+	}
+	return nil
+
+}
+
+//check cluster invoker is destroyed or not
+func (invoker *baseClusterInvoker) checkWhetherDestroyed() error {
+	if invoker.destroyed.IsSet() {
+		ip, _ := gxnet.GetLocalIP()
+		return jerrors.Errorf("Rpc cluster invoker for %v on consumer %v use dubbo version %v is now destroyed! can not invoke any more. ",
+			invoker.directory.GetUrl().Service(), ip, version.Version)
+	}
+	return nil
+}
+
+func (invoker *baseClusterInvoker) doSelect(lb cluster.LoadBalance, invocation protocol.Invocation, invokers []protocol.Invoker, invoked []protocol.Invoker) protocol.Invoker {
+	//todo:ticky connect 粘纸连接
+	if len(invokers) == 1 {
+		return invokers[0]
+	}
+	selectedInvoker := lb.Select(invokers, invoker.GetUrl(), invocation)
+
+	//judge to if the selectedInvoker is invoked
+
+	if !selectedInvoker.IsAvailable() || !invoker.availablecheck || isInvoked(selectedInvoker, invoked) {
+		// do reselect
+		var reslectInvokers []protocol.Invoker
+
+		for _, invoker := range invokers {
+			if !invoker.IsAvailable() {
+				continue
+			}
+
+			if !isInvoked(invoker, invoked) {
+				reslectInvokers = append(reslectInvokers, invoker)
+			}
+		}
+
+		if len(reslectInvokers) > 0 {
+			return lb.Select(reslectInvokers, invoker.GetUrl(), invocation)
+		} else {
+			return nil
+		}
+	}
+	return selectedInvoker
+
+}
+
+func isInvoked(selectedInvoker protocol.Invoker, invoked []protocol.Invoker) bool {
+	if len(invoked) > 0 {
+		for _, i := range invoked {
+			if i == selectedInvoker {
+				return true
+			}
+		}
+	}
+	return false
+}
diff --git a/cluster/support/failover_cluster.go b/cluster/support/failover_cluster.go
new file mode 100644
index 0000000000000000000000000000000000000000..08ffff193c5ca7fc7ce830ab458737b6cc8ba2ee
--- /dev/null
+++ b/cluster/support/failover_cluster.go
@@ -0,0 +1,24 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type FailoverCluster struct {
+}
+
+const name = "failover"
+
+func init() {
+	extension.SetCluster(name, NewFailoverCluster)
+}
+
+func NewFailoverCluster() cluster.Cluster {
+	return &FailoverCluster{}
+}
+
+func (cluster *FailoverCluster) Join(directory cluster.Directory) protocol.Invoker {
+	return newFailoverClusterInvoker(directory)
+}
diff --git a/cluster/support/failover_cluster_invoker.go b/cluster/support/failover_cluster_invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..00b14296e53486c23a28d6764b62658a317ccb3a
--- /dev/null
+++ b/cluster/support/failover_cluster_invoker.go
@@ -0,0 +1,85 @@
+package cluster
+
+import (
+	gxnet "github.com/AlexStocks/goext/net"
+	jerrors "github.com/juju/errors"
+)
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/version"
+)
+
+type failoverClusterInvoker struct {
+	baseClusterInvoker
+}
+
+func newFailoverClusterInvoker(directory cluster.Directory) protocol.Invoker {
+	return &failoverClusterInvoker{
+		baseClusterInvoker: newBaseClusterInvoker(directory),
+	}
+}
+
+func (invoker *failoverClusterInvoker) Invoke(invocation protocol.Invocation) protocol.Result {
+
+	invokers := invoker.directory.List(invocation)
+	err := invoker.checkInvokers(invokers, invocation)
+
+	if err != nil {
+		return &protocol.RPCResult{Err: err}
+	}
+	url := invokers[0].GetUrl()
+
+	methodName := invocation.MethodName()
+	//Get the service loadbalance config
+	lb := url.GetParam(constant.LOADBALANCE_KEY, constant.DEFAULT_LOADBALANCE)
+
+	//Get the service method loadbalance config if have
+	if v := url.GetMethodParam(methodName, constant.LOADBALANCE_KEY, constant.DEFAULT_LOADBALANCE); v != "" {
+		lb = v
+	}
+	loadbalance := extension.GetLoadbalance(lb)
+
+	//get reties
+	retries := url.GetParamInt(constant.RETRIES_KEY, constant.DEFAULT_RETRIES)
+
+	//Get the service method loadbalance config if have
+	if v := url.GetMethodParamInt(methodName, constant.RETRIES_KEY, constant.DEFAULT_RETRIES); v != 0 {
+		retries = v
+	}
+	invoked := []protocol.Invoker{}
+	providers := []string{}
+	var result protocol.Result
+	for i := int64(0); i < retries; i++ {
+		//Reselect before retry to avoid a change of candidate `invokers`.
+		//NOTE: if `invokers` changed, then `invoked` also lose accuracy.
+		if i > 0 {
+			err := invoker.checkWhetherDestroyed()
+			if err != nil {
+				return &protocol.RPCResult{Err: err}
+			}
+			invokers = invoker.directory.List(invocation)
+			err = invoker.checkInvokers(invokers, invocation)
+			if err != nil {
+				return &protocol.RPCResult{Err: err}
+			}
+		}
+		ivk := invoker.doSelect(loadbalance, invocation, invokers, invoked)
+		invoked = append(invoked, ivk)
+		//DO INVOKE
+		result = ivk.Invoke(invocation)
+		if result.Error() != nil {
+			providers = append(providers, ivk.GetUrl().Key())
+			continue
+		} else {
+			return result
+		}
+	}
+	ip, _ := gxnet.GetLocalIP()
+	return &protocol.RPCResult{Err: jerrors.Errorf("Failed to invoke the method %v in the service %v . Tried %v times of "+
+		"the providers %v (%v/%v)from the registry %v on the consumer %v using the dubbo version %v. Last error is %v.",
+		methodName, invoker.GetUrl().Service(), retries, providers, len(providers), len(invokers), invoker.directory.GetUrl(), ip, version.Version, result.Error().Error(),
+	)}
+}
diff --git a/cluster/support/mock_cluster.go b/cluster/support/mock_cluster.go
new file mode 100644
index 0000000000000000000000000000000000000000..58d7465527fdb2fbe0f6af8373feee90541b1b2f
--- /dev/null
+++ b/cluster/support/mock_cluster.go
@@ -0,0 +1,18 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type MockCluster struct {
+}
+
+func NewMockCluster() cluster.Cluster {
+	return &MockCluster{}
+}
+
+func (cluster *MockCluster) Join(directory cluster.Directory) protocol.Invoker {
+	return protocol.NewBaseInvoker(config.URL{})
+}
diff --git a/cluster/support/registry_aware_cluster.go b/cluster/support/registry_aware_cluster.go
new file mode 100644
index 0000000000000000000000000000000000000000..1f2e030d5840cebdcdfb4743875bcaff42c949e8
--- /dev/null
+++ b/cluster/support/registry_aware_cluster.go
@@ -0,0 +1,22 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type RegistryAwareCluster struct {
+}
+
+func init() {
+	extension.SetCluster("registryAware", NewRegistryAwareCluster)
+}
+
+func NewRegistryAwareCluster() cluster.Cluster {
+	return &RegistryAwareCluster{}
+}
+
+func (cluster *RegistryAwareCluster) Join(directory cluster.Directory) protocol.Invoker {
+	return newRegistryAwareClusterInvoker(directory)
+}
diff --git a/cluster/support/registry_aware_cluster_invoker.go b/cluster/support/registry_aware_cluster_invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..2ceef25aeca7568578ea32e90f9014e611cde6e1
--- /dev/null
+++ b/cluster/support/registry_aware_cluster_invoker.go
@@ -0,0 +1,35 @@
+package cluster
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type registryAwareClusterInvoker struct {
+	baseClusterInvoker
+}
+
+func newRegistryAwareClusterInvoker(directory cluster.Directory) protocol.Invoker {
+	return &registryAwareClusterInvoker{
+		baseClusterInvoker: newBaseClusterInvoker(directory),
+	}
+}
+
+func (invoker *registryAwareClusterInvoker) Invoke(invocation protocol.Invocation) protocol.Result {
+	invokers := invoker.directory.List(invocation)
+	//First, pick the invoker (XXXClusterInvoker) that comes from the local registry, distinguish by a 'default' key.
+	for _, invoker := range invokers {
+		if invoker.IsAvailable() && invoker.GetUrl().GetParam(constant.REGISTRY_DEFAULT_KEY, "false") == "true" {
+			return invoker.Invoke(invocation)
+		}
+	}
+
+	//If none of the invokers has a local signal, pick the first one available.
+	for _, invoker := range invokers {
+		if invoker.IsAvailable() {
+			return invoker.Invoke(invocation)
+		}
+	}
+	return nil
+}
diff --git a/common/constant/default.go b/common/constant/default.go
new file mode 100644
index 0000000000000000000000000000000000000000..5871e3cec8ec8dd1c17cb79f002fd9c386d15ecb
--- /dev/null
+++ b/common/constant/default.go
@@ -0,0 +1,19 @@
+package constant
+
+const (
+	DEFAULT_WEIGHT = 100 //
+	DEFAULT_WARMUP = 10 * 60 * 1000
+)
+
+const (
+	DEFAULT_LOADBALANCE = "random"
+	DEFAULT_RETRIES     = 2
+	DEFAULT_PROTOCOL    = "dubbo"
+	DEFAULT_VERSION     = ""
+	DEFAULT_REG_TIMEOUT = "10s"
+	DEFAULT_CLUSTER     = "failover"
+)
+
+const (
+	ECHO = "Echo"
+)
diff --git a/common/constant/env.go b/common/constant/env.go
new file mode 100644
index 0000000000000000000000000000000000000000..4f03e83080c36fcca47d4df1cad5311a30365066
--- /dev/null
+++ b/common/constant/env.go
@@ -0,0 +1,7 @@
+package constant
+
+const (
+	CONF_CONSUMER_FILE_PATH        = "CONF_CONSUMER_FILE_PATH"
+	CONF_PROVIDER_FILE_PATH        = "CONF_PROVIDER_FILE_PATH"
+	APP_LOG_CONF_FILE       string = "APP_LOG_CONF_FILE"
+)
diff --git a/common/constant/key.go b/common/constant/key.go
new file mode 100644
index 0000000000000000000000000000000000000000..c0a009bd4c62fb62ddf14869d635d4e15b46922b
--- /dev/null
+++ b/common/constant/key.go
@@ -0,0 +1,52 @@
+package constant
+
+const (
+	ASYNC_KEY = "async" // it's value should be "true" or "false" of string type
+)
+
+const (
+	GROUP_KEY     = "group"
+	VERSION_KEY   = "version"
+	INTERFACE_KEY = "interface"
+	PATH_KEY      = "path"
+	SERVICE_KEY   = "service"
+	METHODS_KEY   = "methods"
+	TIMEOUT_KEY   = "timeout"
+)
+
+const (
+	SERVICE_FILTER_KEY   = "service.filter"
+	REFERENCE_FILTER_KEY = "reference.filter"
+)
+
+const (
+	TIMESTAMP_KEY        = "timestamp"
+	REMOTE_TIMESTAMP_KEY = "remote.timestamp"
+	CLUSTER_KEY          = "cluster"
+	LOADBALANCE_KEY      = "loadbalance"
+	WEIGHT_KEY           = "weight"
+	WARMUP_KEY           = "warmup"
+	RETRIES_KEY          = "retries"
+)
+
+const (
+	DUBBOGO_CTX_KEY = "dubbogo-ctx"
+)
+
+const (
+	REGISTRY_KEY         = "registry"
+	REGISTRY_PROTOCOL    = "registry"
+	ROLE_KEY             = "registry.role"
+	REGISTRY_DEFAULT_KEY = "registry.default"
+	REGISTRY_TIMEOUT_KEY = "registry.timeout"
+)
+
+const (
+	APPLICATION_KEY  = "application"
+	ORGANIZATION_KEY = "organization"
+	NAME_KEY         = "name"
+	MODULE_KEY       = "module"
+	APP_VERSION_KEY  = "app.version"
+	OWNER_KEY        = "owner"
+	ENVIRONMENT_KEY  = "environment"
+)
diff --git a/common/extension/cluster.go b/common/extension/cluster.go
new file mode 100644
index 0000000000000000000000000000000000000000..1fd76a4756fbce1a198e6ea410acb052cd8dc537
--- /dev/null
+++ b/common/extension/cluster.go
@@ -0,0 +1,17 @@
+package extension
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster"
+)
+
+var (
+	clusters = make(map[string]func() cluster.Cluster)
+)
+
+func SetCluster(name string, fcn func() cluster.Cluster) {
+	clusters[name] = fcn
+}
+
+func GetCluster(name string) cluster.Cluster {
+	return clusters[name]()
+}
diff --git a/common/extension/filter.go b/common/extension/filter.go
new file mode 100644
index 0000000000000000000000000000000000000000..e602916ecadb2e65df47529e09ba0ffaa59dfc30
--- /dev/null
+++ b/common/extension/filter.go
@@ -0,0 +1,21 @@
+package extension
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/filter"
+)
+
+var (
+	filters map[string]func() filter.Filter
+)
+
+func init() {
+	filters = make(map[string]func() filter.Filter)
+}
+
+func SetFilter(name string, v func() filter.Filter) {
+	filters[name] = v
+}
+
+func GetFilterExtension(name string) filter.Filter {
+	return filters[name]()
+}
diff --git a/common/extension/loadbalance.go b/common/extension/loadbalance.go
new file mode 100644
index 0000000000000000000000000000000000000000..ae1998fb3f404cc5c8d3f43058657c8b9328da8a
--- /dev/null
+++ b/common/extension/loadbalance.go
@@ -0,0 +1,15 @@
+package extension
+
+import "github.com/dubbo/go-for-apache-dubbo/cluster"
+
+var (
+	loadbalances = make(map[string]func() cluster.LoadBalance)
+)
+
+func SetLoadbalance(name string, fcn func() cluster.LoadBalance) {
+	loadbalances[name] = fcn
+}
+
+func GetLoadbalance(name string) cluster.LoadBalance {
+	return loadbalances[name]()
+}
diff --git a/common/extension/protocol.go b/common/extension/protocol.go
new file mode 100644
index 0000000000000000000000000000000000000000..2cc14abf82eca77d8796b8708d2ffa56eba2224f
--- /dev/null
+++ b/common/extension/protocol.go
@@ -0,0 +1,21 @@
+package extension
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+var (
+	protocols map[string]func() protocol.Protocol
+)
+
+func init() {
+	protocols = make(map[string]func() protocol.Protocol)
+}
+
+func SetProtocol(name string, v func() protocol.Protocol) {
+	protocols[name] = v
+}
+
+func GetProtocolExtension(name string) protocol.Protocol {
+	return protocols[name]()
+}
diff --git a/common/extension/registry.go b/common/extension/registry.go
new file mode 100644
index 0000000000000000000000000000000000000000..82291c84b37dea6a62f913bbdd37d3ed43ec17a4
--- /dev/null
+++ b/common/extension/registry.go
@@ -0,0 +1,27 @@
+package extension
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
+)
+
+var (
+	registrys map[string]func(config *config.URL) (registry.Registry, error)
+)
+
+/*
+it must excute first
+*/
+func init() {
+	// init map
+	registrys = make(map[string]func(config *config.URL) (registry.Registry, error))
+}
+
+func SetRegistry(name string, v func(config *config.URL) (registry.Registry, error)) {
+	registrys[name] = v
+}
+
+func GetRegistryExtension(name string, config *config.URL) (registry.Registry, error) {
+	return registrys[name](config)
+
+}
diff --git a/common/node.go b/common/node.go
new file mode 100644
index 0000000000000000000000000000000000000000..4090f8cdacbe84c5fcd89611a1d987a4482171d1
--- /dev/null
+++ b/common/node.go
@@ -0,0 +1,9 @@
+package common
+
+import "github.com/dubbo/go-for-apache-dubbo/config"
+
+type Node interface {
+	GetUrl() config.URL
+	IsAvailable() bool
+	Destroy()
+}
diff --git a/common/proxy/proxy.go b/common/proxy/proxy.go
new file mode 100644
index 0000000000000000000000000000000000000000..6d88c5fc8aa2ec2c45597c55f2e10aa7112941fd
--- /dev/null
+++ b/common/proxy/proxy.go
@@ -0,0 +1,118 @@
+package proxy
+
+import (
+	"reflect"
+)
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/support"
+)
+
+// Proxy struct
+type Proxy struct {
+	rpc         config.RPCService
+	invoke      protocol.Invoker
+	callBack    interface{}
+	attachments map[string]string
+}
+
+var typError = reflect.Zero(reflect.TypeOf((*error)(nil)).Elem()).Type()
+
+func NewProxy(invoke protocol.Invoker, callBack interface{}, attachments map[string]string) *Proxy {
+	return &Proxy{
+		invoke:      invoke,
+		callBack:    callBack,
+		attachments: attachments,
+	}
+}
+
+// proxy implement
+// In consumer, RPCService like:
+// 		type XxxProvider struct {
+//  		Yyy func(ctx context.Context, args []interface{}, rsp *Zzz) error
+// 		}
+func (p *Proxy) Implement(v config.RPCService) {
+
+	// check parameters, incoming interface must be a elem's pointer.
+	valueOf := reflect.ValueOf(v)
+	log.Debug("[Implement] reflect.TypeOf: %s", valueOf.String())
+
+	valueOfElem := valueOf.Elem()
+	typeOf := valueOfElem.Type()
+
+	// check incoming interface, incoming interface's elem must be a struct.
+	if typeOf.Kind() != reflect.Struct {
+		log.Error("%s must be a struct ptr", valueOf.String())
+		return
+	}
+
+	makeDubboCallProxy := func(methodName string, outs []reflect.Type) func(in []reflect.Value) []reflect.Value {
+		return func(in []reflect.Value) []reflect.Value {
+
+			inv := support.NewRPCInvocationForConsumer(methodName, nil, in[1].Interface().([]interface{}), in[2].Interface(), p.callBack, config.URL{}, nil)
+			for k, v := range p.attachments {
+				inv.SetAttachments(k, v)
+			}
+
+			result := p.invoke.Invoke(inv)
+			var err error
+			err = result.Error()
+			log.Info("[makeDubboCallProxy] err: %v", err)
+			return []reflect.Value{reflect.ValueOf(&err).Elem()}
+		}
+	}
+
+	numField := valueOfElem.NumField()
+	for i := 0; i < numField; i++ {
+		t := typeOf.Field(i)
+		f := valueOfElem.Field(i)
+		if f.Kind() == reflect.Func && f.IsValid() && f.CanSet() {
+
+			if t.Type.NumIn() != 3 && t.Type.NumIn() != 4 {
+				log.Warn("method %s of mtype %v has wrong number of in parameters %d; needs exactly 3/4",
+					t.Name, t.Type.String(), t.Type.NumIn())
+				continue
+			}
+
+			if t.Type.NumIn() == 3 && t.Type.In(2).Kind() != reflect.Ptr {
+				log.Warn("reply type of method %q is not a pointer %v", t.Name, t.Type.In(2))
+				continue
+			}
+
+			if t.Type.NumIn() == 4 && t.Type.In(3).Kind() != reflect.Ptr {
+				log.Warn("reply type of method %q is not a pointer %v", t.Name, t.Type.In(3))
+				continue
+			}
+
+			// Method needs one out.
+			if t.Type.NumOut() != 1 {
+				log.Warn("method %q has %d out parameters; needs exactly 1", t.Name, t.Type.NumOut())
+				continue
+			}
+			// The return type of the method must be error.
+			if returnType := t.Type.Out(0); returnType != typError {
+				log.Warn("return type %s of method %q is not error", returnType, t.Name)
+				continue
+			}
+
+			var funcOuts = make([]reflect.Type, t.Type.NumOut())
+			funcOuts[0] = t.Type.Out(0)
+
+			// do method proxy here:
+			f.Set(reflect.MakeFunc(f.Type(), makeDubboCallProxy(t.Name, funcOuts)))
+			log.Debug("set method [%s]", t.Name)
+		}
+	}
+
+	p.rpc = v
+
+}
+
+func (p *Proxy) Get() config.RPCService {
+	return p.rpc
+}
diff --git a/common/proxy/proxy_test.go b/common/proxy/proxy_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..10a051652b5e749ccbfe3f4f70ee63d142230ccc
--- /dev/null
+++ b/common/proxy/proxy_test.go
@@ -0,0 +1,94 @@
+package proxy
+
+import (
+	"context"
+	"errors"
+	"testing"
+)
+
+import (
+	"github.com/stretchr/testify/assert"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type TestService struct {
+	MethodOne func(context.Context, []interface{}, *struct{}) error
+}
+
+func (s *TestService) Service() string {
+	return "com.test.Path"
+}
+func (s *TestService) Version() string {
+	return ""
+}
+
+func TestProxy_Implement(t *testing.T) {
+
+	invoker := protocol.NewBaseInvoker(config.URL{})
+	p := NewProxy(invoker, nil, nil)
+	s := &TestService{MethodOne: func(i context.Context, i2 []interface{}, i3 *struct{}) error {
+		return errors.New("errors")
+	}}
+	p.Implement(s)
+	err := s.MethodOne(nil, nil, nil)
+	assert.NoError(t, err)
+
+	// inherit & lowercase
+	type S1 struct {
+		TestService
+		methodOne func(context.Context, []interface{}, *struct{}) error
+	}
+	s1 := &S1{TestService: *s, methodOne: func(i context.Context, i2 []interface{}, i3 *struct{}) error {
+		return errors.New("errors")
+	}}
+	p.Implement(s1)
+	err = s1.MethodOne(nil, nil, nil)
+	assert.NoError(t, err)
+	err = s1.methodOne(nil, nil, nil)
+	assert.EqualError(t, err, "errors")
+
+	// args number
+	type S2 struct {
+		TestService
+		MethodOne func(context.Context, []interface{}) error
+	}
+	s2 := &S2{TestService: *s}
+	p.Implement(s2)
+	assert.Nil(t, s2.MethodOne)
+	//assert.EqualError(t, err, "method MethodOne of mtype func(context.Context, []interface {}) error has wrong number of in parameters 2; needs exactly 3/4")
+
+	// returns number
+	type S3 struct {
+		TestService
+		MethodOne func(context.Context, []interface{}, *struct{}) (interface{}, error)
+	}
+	s3 := &S3{TestService: *s}
+	p.Implement(s3)
+	assert.Nil(t, s3.MethodOne)
+	//assert.EqualError(t, err, "method \"MethodOne\" has 2 out parameters; needs exactly 1")
+
+	// returns type
+	type S4 struct {
+		TestService
+		MethodOne func(context.Context, []interface{}, *struct{}) interface{}
+	}
+	s4 := &S4{TestService: *s}
+	p.Implement(s4)
+	assert.Nil(t, s4.MethodOne)
+	//assert.EqualError(t, err, "return type interface {} of method \"MethodOne\" is not error")
+
+	// reply type
+	type S5 struct {
+		TestService
+		MethodOne func(context.Context, []interface{}, interface{}) error
+	}
+	s5 := &S5{TestService: *s}
+	p.Implement(s5)
+	assert.Nil(t, s5.MethodOne)
+	//assert.EqualError(t, err, "reply type of method \"MethodOne\" is not a pointer interface {}")
+
+}
diff --git a/config/rpc_service.go b/config/rpc_service.go
new file mode 100644
index 0000000000000000000000000000000000000000..d3607775305e23a9234b882898bc576452114fcc
--- /dev/null
+++ b/config/rpc_service.go
@@ -0,0 +1,234 @@
+package config
+
+import (
+	"context"
+	"reflect"
+	"strings"
+	"sync"
+	"unicode"
+	"unicode/utf8"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+)
+
+// rpc service interface
+type RPCService interface {
+	Service() string // Path InterfaceName
+	Version() string
+}
+
+var (
+	// Precompute the reflect type for error. Can't use error directly
+	// because Typeof takes an empty interface value. This is annoying.
+	typeOfError = reflect.TypeOf((*error)(nil)).Elem()
+
+	ServiceMap = &serviceMap{
+		serviceMap: make(map[string]map[string]*Service),
+	}
+)
+
+//////////////////////////
+// info of method
+//////////////////////////
+
+type MethodType struct {
+	method    reflect.Method
+	ctxType   reflect.Type // type of the request context
+	argType   reflect.Type
+	replyType reflect.Type
+}
+
+func (m *MethodType) Method() reflect.Method {
+	return m.method
+}
+func (m *MethodType) CtxType() reflect.Type {
+	return m.ctxType
+}
+func (m *MethodType) ArgType() reflect.Type {
+	return m.argType
+}
+func (m *MethodType) ReplyType() reflect.Type {
+	return m.replyType
+}
+func (m *MethodType) SuiteContext(ctx context.Context) reflect.Value {
+	if contextv := reflect.ValueOf(ctx); contextv.IsValid() {
+		return contextv
+	}
+	return reflect.Zero(m.ctxType)
+}
+
+//////////////////////////
+// info of service interface
+//////////////////////////
+
+type Service struct {
+	name     string
+	rcvr     reflect.Value
+	rcvrType reflect.Type
+	methods  map[string]*MethodType
+}
+
+func (s *Service) Method() map[string]*MethodType {
+	return s.methods
+}
+func (s *Service) RcvrType() reflect.Type {
+	return s.rcvrType
+}
+func (s *Service) Rcvr() reflect.Value {
+	return s.rcvr
+}
+
+//////////////////////////
+// serviceMap
+//////////////////////////
+
+type serviceMap struct {
+	mutex      sync.RWMutex                   // protects the serviceMap
+	serviceMap map[string]map[string]*Service // protocol -> service name -> service
+}
+
+func (sm *serviceMap) GetService(protocol, name string) *Service {
+	sm.mutex.RLock()
+	defer sm.mutex.RUnlock()
+	if s, ok := sm.serviceMap[protocol]; ok {
+		if srv, ok := s[name]; ok {
+			return srv
+		}
+		return nil
+	}
+	return nil
+}
+
+func (sm *serviceMap) Register(protocol string, rcvr RPCService) (string, error) {
+	if sm.serviceMap[protocol] == nil {
+		sm.serviceMap[protocol] = make(map[string]*Service)
+	}
+
+	s := new(Service)
+	s.rcvrType = reflect.TypeOf(rcvr)
+	s.rcvr = reflect.ValueOf(rcvr)
+	sname := reflect.Indirect(s.rcvr).Type().Name()
+	if sname == "" {
+		s := "no service name for type " + s.rcvrType.String()
+		log.Error(s)
+		return "", jerrors.New(s)
+	}
+	if !isExported(sname) {
+		s := "type " + sname + " is not exported"
+		log.Error(s)
+		return "", jerrors.New(s)
+	}
+
+	sname = rcvr.Service()
+	if server := sm.GetService(protocol, sname); server != nil {
+		return "", jerrors.New("service already defined: " + sname)
+	}
+	s.name = sname
+	s.methods = make(map[string]*MethodType)
+
+	// Install the methods
+	methods := ""
+	methods, s.methods = suitableMethods(s.rcvrType)
+
+	if len(s.methods) == 0 {
+		s := "type " + sname + " has no exported methods of suitable type"
+		log.Error(s)
+		return "", jerrors.New(s)
+	}
+	sm.mutex.Lock()
+	sm.serviceMap[protocol][s.name] = s
+	sm.mutex.Unlock()
+
+	return strings.TrimSuffix(methods, ","), nil
+}
+
+// Is this an exported - upper case - name
+func isExported(name string) bool {
+	rune, _ := utf8.DecodeRuneInString(name)
+	return unicode.IsUpper(rune)
+}
+
+// Is this type exported or a builtin?
+func isExportedOrBuiltinType(t reflect.Type) bool {
+	for t.Kind() == reflect.Ptr {
+		t = t.Elem()
+	}
+	// PkgPath will be non-empty even for an exported type,
+	// so we need to check the type name as well.
+	return isExported(t.Name()) || t.PkgPath() == ""
+}
+
+// suitableMethods returns suitable Rpc methods of typ
+func suitableMethods(typ reflect.Type) (string, map[string]*MethodType) {
+	methods := make(map[string]*MethodType)
+	mts := ""
+	log.Debug("[%s] NumMethod is %d", typ.String(), typ.NumMethod())
+	for m := 0; m < typ.NumMethod(); m++ {
+		method := typ.Method(m)
+		if mt := suiteMethod(method); mt != nil {
+			methods[method.Name] = mt
+			if m == 0 {
+				mts += method.Name
+			} else {
+				mts += "," + method.Name
+			}
+		}
+	}
+	return mts, methods
+}
+
+// suiteMethod returns a suitable Rpc methodType
+func suiteMethod(method reflect.Method) *MethodType {
+	mtype := method.Type
+	mname := method.Name
+
+	// Method must be exported.
+	if method.PkgPath != "" {
+		return nil
+	}
+
+	var replyType, argType, ctxType reflect.Type
+	switch mtype.NumIn() {
+	case 3:
+		argType = mtype.In(1)
+		replyType = mtype.In(2)
+	case 4:
+		ctxType = mtype.In(1)
+		argType = mtype.In(2)
+		replyType = mtype.In(3)
+	default:
+		log.Error("method %s of mtype %v has wrong number of in parameters %d; needs exactly 3/4",
+			mname, mtype, mtype.NumIn())
+		return nil
+	}
+	// First arg need not be a pointer.
+	if !isExportedOrBuiltinType(argType) {
+		log.Error("argument type of method %q is not exported %v", mname, argType)
+		return nil
+	}
+	// Second arg must be a pointer.
+	if replyType.Kind() != reflect.Ptr {
+		log.Error("reply type of method %q is not a pointer %v", mname, replyType)
+		return nil
+	}
+	// Reply type must be exported.
+	if !isExportedOrBuiltinType(replyType) {
+		log.Error("reply type of method %s not exported{%v}", mname, replyType)
+		return nil
+	}
+	// Method needs one out.
+	if mtype.NumOut() != 1 {
+		log.Error("method %q has %d out parameters; needs exactly 1", mname, mtype.NumOut())
+		return nil
+	}
+	// The return type of the method must be error.
+	if returnType := mtype.Out(0); returnType != typeOfError {
+		log.Error("return type %s of method %q is not error", returnType, mname)
+		return nil
+	}
+
+	return &MethodType{method: method, argType: argType, replyType: replyType, ctxType: ctxType}
+}
diff --git a/config/rpc_service_test.go b/config/rpc_service_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..ce4edf48f39662e2d1f90ab4c8c9abf1a7f4f6b6
--- /dev/null
+++ b/config/rpc_service_test.go
@@ -0,0 +1,36 @@
+package config
+
+import (
+	"context"
+	"testing"
+)
+
+import (
+	"github.com/stretchr/testify/assert"
+)
+
+type TestService struct {
+}
+
+func (s *TestService) MethodOne(ctx context.Context, args []interface{}, rsp *struct{}) error {
+	return nil
+}
+
+func (s *TestService) MethodTwo(ctx context.Context, args []interface{}, rsp *struct{}) error {
+	return nil
+
+}
+
+func (s *TestService) Service() string {
+	return "com.test.Path"
+}
+func (s *TestService) Version() string {
+	return ""
+}
+
+func TestServiceMap_Register(t *testing.T) {
+	s := &TestService{}
+	methods, err := ServiceMap.Register("testporotocol", s)
+	assert.NoError(t, err)
+	assert.Equal(t, "MethodOne,MethodTwo", methods)
+}
diff --git a/config/support/application_config.go b/config/support/application_config.go
new file mode 100644
index 0000000000000000000000000000000000000000..367bc20468c19f4d3e70eb83aa06ab60d7b6688e
--- /dev/null
+++ b/config/support/application_config.go
@@ -0,0 +1,10 @@
+package support
+
+type ApplicationConfig struct {
+	Organization string `yaml:"organization"  json:"organization,omitempty"`
+	Name         string `yaml:"name" json:"name,omitempty"`
+	Module       string `yaml:"module" json:"module,omitempty"`
+	Version      string `yaml:"version" json:"version,omitempty"`
+	Owner        string `yaml:"owner" json:"owner,omitempty"`
+	Environment  string `yaml:"environment" json:"environment,omitempty"`
+}
diff --git a/config/support/config_loader.go b/config/support/config_loader.go
new file mode 100644
index 0000000000000000000000000000000000000000..2563f9779fe5b4c63ef893db589e9dc4f9c129a9
--- /dev/null
+++ b/config/support/config_loader.go
@@ -0,0 +1,245 @@
+package support
+
+import (
+	"fmt"
+	"io/ioutil"
+	"os"
+	"path"
+	"strings"
+	"time"
+)
+
+import (
+	"github.com/AlexStocks/goext/log"
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+	"gopkg.in/yaml.v2"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+)
+
+var (
+	consumerConfig *ConsumerConfig
+	providerConfig *ProviderConfig
+)
+
+// loaded comsumer & provider config from xxx.yml, and log config from xxx.xml
+// Namely: dubbo.comsumer.xml & dubbo.provider.xml in java dubbo
+func init() {
+
+	if err := logInit(); err != nil { // log config
+		log.Warn("[logInit] %#v", err)
+	}
+
+	var (
+		confConFile, confProFile string
+	)
+
+	confConFile = os.Getenv(constant.CONF_CONSUMER_FILE_PATH)
+	confProFile = os.Getenv(constant.CONF_PROVIDER_FILE_PATH)
+
+	if errCon := consumerInit(confConFile); errCon != nil {
+		log.Warn("[consumerInit] %#v", errCon)
+		consumerConfig = nil
+	}
+	if errPro := providerInit(confProFile); errPro != nil {
+		log.Warn("[providerInit] %#v", errPro)
+		providerConfig = nil
+	}
+
+}
+
+func logInit() error {
+	var (
+		confFile string
+	)
+
+	confFile = os.Getenv(constant.APP_LOG_CONF_FILE)
+	if confFile == "" {
+		return fmt.Errorf("log configure file name is nil")
+	}
+	if path.Ext(confFile) != ".xml" {
+		return fmt.Errorf("log configure file name{%v} suffix must be .xml", confFile)
+	}
+
+	log.LoadConfiguration(confFile)
+
+	return nil
+}
+
+func consumerInit(confConFile string) error {
+	if confConFile == "" {
+		return fmt.Errorf("application configure(consumer) file name is nil")
+	}
+
+	if path.Ext(confConFile) != ".yml" {
+		return fmt.Errorf("application configure file name{%v} suffix must be .yml", confConFile)
+	}
+
+	confFileStream, err := ioutil.ReadFile(confConFile)
+	if err != nil {
+		return fmt.Errorf("ioutil.ReadFile(file:%s) = error:%s", confConFile, jerrors.ErrorStack(err))
+	}
+	consumerConfig = &ConsumerConfig{}
+	err = yaml.Unmarshal(confFileStream, consumerConfig)
+	if err != nil {
+		return fmt.Errorf("yaml.Unmarshal() = error:%s", jerrors.ErrorStack(err))
+	}
+
+	gxlog.CInfo("consumer config{%#v}\n", consumerConfig)
+	return nil
+}
+
+func providerInit(confProFile string) error {
+	if confProFile == "" {
+		return fmt.Errorf("application configure(provider) file name is nil")
+	}
+
+	if path.Ext(confProFile) != ".yml" {
+		return fmt.Errorf("application configure file name{%v} suffix must be .yml", confProFile)
+	}
+
+	confFileStream, err := ioutil.ReadFile(confProFile)
+	if err != nil {
+		return fmt.Errorf("ioutil.ReadFile(file:%s) = error:%s", confProFile, jerrors.ErrorStack(err))
+	}
+	providerConfig = &ProviderConfig{}
+	err = yaml.Unmarshal(confFileStream, providerConfig)
+	if err != nil {
+		return fmt.Errorf("yaml.Unmarshal() = error:%s", jerrors.ErrorStack(err))
+	}
+
+	//todo: provider config
+
+	gxlog.CInfo("provider config{%#v}\n", providerConfig)
+	return nil
+}
+
+/////////////////////////
+// consumerConfig
+/////////////////////////
+
+type ConsumerConfig struct {
+	// pprof
+	Pprof_Enabled bool `default:"false" yaml:"pprof_enabled" json:"pprof_enabled,omitempty"`
+	Pprof_Port    int  `default:"10086"  yaml:"pprof_port" json:"pprof_port,omitempty"`
+
+	// client
+	Connect_Timeout string `default:"100ms"  yaml:"connect_timeout" json:"connect_timeout,omitempty"`
+	ConnectTimeout  time.Duration
+
+	Request_Timeout string `yaml:"request_timeout" default:"5s" json:"request_timeout,omitempty"` // 500ms, 1m
+	RequestTimeout  time.Duration
+
+	// codec & selector & transport & registry
+	Selector     string `default:"cache"  yaml:"selector" json:"selector,omitempty"`
+	Selector_TTL string `default:"10m"  yaml:"selector_ttl" json:"selector_ttl,omitempty"`
+	// application
+	ApplicationConfig ApplicationConfig `yaml:"application_config" json:"application_config,omitempty"`
+	Registries        []RegistryConfig  `yaml:"registries" json:"registries,omitempty"`
+	References        []ReferenceConfig `yaml:"references" json:"references,omitempty"`
+}
+
+type ReferenceConfigTmp struct {
+	Service    string           `required:"true"  yaml:"service"  json:"service,omitempty"`
+	Registries []RegistryConfig `required:"true"  yaml:"registries"  json:"registries,omitempty"`
+	URLs       []map[string]string
+}
+
+func SetConsumerConfig(c ConsumerConfig) {
+	consumerConfig = &c
+}
+func GetConsumerConfig() ConsumerConfig {
+	if consumerConfig == nil {
+		log.Warn("consumerConfig is nil!")
+		return ConsumerConfig{}
+	}
+	return *consumerConfig
+}
+
+/////////////////////////
+// providerConfig
+/////////////////////////
+
+type ProviderConfig struct {
+	// pprof
+	Pprof_Enabled bool `default:"false" yaml:"pprof_enabled" json:"pprof_enabled,omitempty"`
+	Pprof_Port    int  `default:"10086"  yaml:"pprof_port" json:"pprof_port,omitempty"`
+
+	ApplicationConfig ApplicationConfig `yaml:"application_config" json:"application_config,omitempty"`
+	Path              string            `yaml:"path" json:"path,omitempty"`
+	Registries        []RegistryConfig  `yaml:"registries" json:"registries,omitempty"`
+	Services          []ServiceConfig   `yaml:"services" json:"services,omitempty"`
+	Protocols         []ProtocolConfig  `yaml:"protocols" json:"protocols,omitempty"`
+}
+
+func SetProviderConfig(p ProviderConfig) {
+	providerConfig = &p
+}
+func GetProviderConfig() ProviderConfig {
+	if providerConfig == nil {
+		log.Warn("providerConfig is nil!")
+		return ProviderConfig{}
+	}
+	return *providerConfig
+}
+
+type ProtocolConfig struct {
+	Name        string `required:"true" yaml:"name"  json:"name,omitempty"`
+	Ip          string `required:"true" yaml:"ip"  json:"ip,omitempty"`
+	Port        string `required:"true" yaml:"port"  json:"port,omitempty"`
+	ContextPath string `required:"true" yaml:"contextPath"  json:"contextPath,omitempty"`
+}
+
+func loadProtocol(protocolsIds string, protocols []ProtocolConfig) []ProtocolConfig {
+	returnProtocols := []ProtocolConfig{}
+	for _, v := range strings.Split(protocolsIds, ",") {
+		for _, prot := range protocols {
+			if v == prot.Name {
+				returnProtocols = append(returnProtocols, prot)
+			}
+		}
+
+	}
+	return returnProtocols
+}
+
+// Dubbo Init
+func Load() (map[string]*ReferenceConfig, map[string]*ServiceConfig) {
+	var refMap map[string]*ReferenceConfig
+	var srvMap map[string]*ServiceConfig
+
+	// reference config
+	if consumerConfig == nil {
+		log.Warn("consumerConfig is nil!")
+	} else {
+		refMap = make(map[string]*ReferenceConfig)
+		length := len(consumerConfig.References)
+		for index := 0; index < length; index++ {
+			con := &consumerConfig.References[index]
+			con.Refer()
+			con.Implement(conServices[con.InterfaceName])
+			refMap[con.InterfaceName] = con
+		}
+	}
+
+	// service config
+	if providerConfig == nil {
+		log.Warn("providerConfig is nil!")
+	} else {
+		srvMap = make(map[string]*ServiceConfig)
+		length := len(providerConfig.Services)
+		for index := 0; index < length; index++ {
+			pro := &providerConfig.Services[index]
+			pro.Implement(proServices[pro.InterfaceName])
+			if err := pro.Export(); err != nil {
+				panic(fmt.Sprintf("service %s export failed! ", pro.InterfaceName))
+			}
+			srvMap[pro.InterfaceName] = pro
+		}
+	}
+
+	return refMap, srvMap
+}
diff --git a/config/support/consumer_config.yml b/config/support/consumer_config.yml
new file mode 100644
index 0000000000000000000000000000000000000000..e98c3a4303347c8a5474effdfd2da9c28d5115a1
--- /dev/null
+++ b/config/support/consumer_config.yml
@@ -0,0 +1,47 @@
+# dubbo client yaml configure file
+
+# pprof
+pprof_enabled : true
+pprof_port : 10086
+
+# client
+request_timeout : "3500ms"
+net_io_timeout : "2s"
+# connect timeout
+connect_timeout : "100ms"
+
+# application config
+application_config:
+    organization : "ikurento.com"
+    name  : "BDTService"
+    module : "dubbogo user-info client"
+    version : "0.0.1"
+    owner : "ZX"
+    environment : "dev"
+
+registries :
+  - id: "hangzhouzk"
+    type: "zookeeper"
+    timeout	: "3s"
+    address: "127.0.0.1:2181"
+    username: ""
+    password: ""
+#    default: "true"
+
+  - id: "shanghaizk"
+    type: "zookeeper"
+    timeout	: "3s"
+    address: "127.0.0.1:2181"
+    username: ""
+    password: ""
+
+references:
+  - registries :
+      - "hangzhouzk"
+      - "shanghaizk"
+#    protocol : "dubbo"
+    interface : "com.ikurento.user.UserProvider"
+    cluster: "failover"
+    methods :
+      - name: "GetUser"
+        retries: 3
diff --git a/config/support/provider_config.yml b/config/support/provider_config.yml
new file mode 100644
index 0000000000000000000000000000000000000000..f6a4c593b04e2cda57dd422fa8f8ccfbf27272fa
--- /dev/null
+++ b/config/support/provider_config.yml
@@ -0,0 +1,60 @@
+# dubbo server yaml configure file
+
+# pprof
+pprof_enabled : true
+pprof_port : 20080
+
+# server
+transport : "http"
+net_timeout : "3s"
+
+path: ""
+# application config
+application_config:
+    organization : "ikurento.com"
+    name : "BDTService"
+    module : "dubbogo user-info server"
+    version : "0.0.1"
+    owner : "ZX"
+    environment : "dev"
+
+registries :
+    - id: "hangzhouzk"
+      type: "zookeeper"
+      timeout	: "3s"
+      address: "127.0.0.1:2181"
+      username: ""
+      password: ""
+
+    - id: "shanghaizk"
+      type: "zookeeper"
+      timeout	: "3s"
+      address: "127.0.0.1:2181"
+      username: ""
+      password: ""
+
+
+services:
+      - registries:
+          - "hangzhouzk"
+          - "shanghaizk"
+        protocol : "dubbo,jsonrpc"
+        # 相当于dubbo.xml中的interface
+        interface : "com.ikurento.user.UserProvider"
+        loadbalance: "random"
+        warmup: "100"
+        cluster: "failover"
+        methods:
+            - name: "GetUser"
+              retries: 1
+              loadbalance: "random"
+
+protocols:
+    -   name: "dubbo"
+        # while using dubbo protocol, ip cannot is 127.0.0.1, because client of java-dubbo will get 'connection refuse'
+        ip : "192.168.56.1"
+        port : 20000
+    -   name: "jsonrpc"
+        ip: "127.0.0.1"
+        port: 20001
+
diff --git a/config/support/reference_config.go b/config/support/reference_config.go
new file mode 100644
index 0000000000000000000000000000000000000000..7c781223f46ef86f54e4d5521650d733d97b7d23
--- /dev/null
+++ b/config/support/reference_config.go
@@ -0,0 +1,111 @@
+package support
+
+import (
+	"context"
+	"net/url"
+	"strconv"
+	"time"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster/directory"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/common/proxy"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type ReferenceConfig struct {
+	context       context.Context
+	pxy           *proxy.Proxy
+	InterfaceName string           `required:"true"  yaml:"interface"  json:"interface,omitempty"`
+	Protocol      string           `yaml:"protocol"  json:"protocol,omitempty"`
+	Registries    []ConfigRegistry `required:"true"  yaml:"registries"  json:"registries,omitempty"`
+	Cluster       string           `yaml:"cluster"  json:"cluster,omitempty"`
+	Loadbalance   string           `yaml:"loadbalance"  json:"loadbalance,omitempty"`
+	Retries       int64            `yaml:"retries"  json:"retries,omitempty"`
+	Group         string           `yaml:"group"  json:"group,omitempty"`
+	Version       string           `yaml:"version"  json:"version,omitempty"`
+	Methods       []struct {
+		Name        string `yaml:"name"  json:"name,omitempty"`
+		Retries     int64  `yaml:"retries"  json:"retries,omitempty"`
+		Loadbalance string `yaml:"loadbalance"  json:"loadbalance,omitempty"`
+	} `yaml:"methods"  json:"methods,omitempty"`
+	async   bool `yaml:"async"  json:"async,omitempty"`
+	invoker protocol.Invoker
+}
+
+type ConfigRegistry string
+
+func NewReferenceConfig(ctx context.Context) *ReferenceConfig {
+	return &ReferenceConfig{context: ctx}
+}
+
+func (refconfig *ReferenceConfig) Refer() {
+	//首先是user specified SubURL, could be peer-to-peer address, or register center's address.
+
+	//其次是assemble SubURL from register center's configuration模式
+	regUrls := loadRegistries(refconfig.Registries, consumerConfig.Registries, config.CONSUMER)
+	url := config.NewURLWithOptions(refconfig.InterfaceName, config.WithProtocol(refconfig.Protocol), config.WithParams(refconfig.getUrlMap()))
+
+	//set url to regUrls
+	for _, regUrl := range regUrls {
+		regUrl.SubURL = url
+	}
+
+	if len(regUrls) == 1 {
+		refconfig.invoker = extension.GetProtocolExtension("registry").Refer(*regUrls[0])
+
+	} else {
+		invokers := []protocol.Invoker{}
+		for _, regUrl := range regUrls {
+			invokers = append(invokers, extension.GetProtocolExtension("registry").Refer(*regUrl))
+		}
+		cluster := extension.GetCluster("registryAware")
+		refconfig.invoker = cluster.Join(directory.NewStaticDirectory(invokers))
+	}
+	//create proxy
+	attachments := map[string]string{}
+	attachments[constant.ASYNC_KEY] = url.GetParam(constant.ASYNC_KEY, "false")
+	refconfig.pxy = proxy.NewProxy(refconfig.invoker, nil, attachments)
+}
+
+// @v is service provider implemented RPCService
+func (refconfig *ReferenceConfig) Implement(v config.RPCService) {
+	refconfig.pxy.Implement(v)
+}
+
+func (refconfig *ReferenceConfig) GetRPCService() config.RPCService {
+	return refconfig.pxy.Get()
+}
+
+func (refconfig *ReferenceConfig) getUrlMap() url.Values {
+	urlMap := url.Values{}
+	urlMap.Set(constant.INTERFACE_KEY, refconfig.InterfaceName)
+	urlMap.Set(constant.TIMESTAMP_KEY, strconv.FormatInt(time.Now().Unix(), 10))
+	urlMap.Set(constant.CLUSTER_KEY, refconfig.Cluster)
+	urlMap.Set(constant.LOADBALANCE_KEY, refconfig.Loadbalance)
+	urlMap.Set(constant.RETRIES_KEY, strconv.FormatInt(refconfig.Retries, 10))
+	urlMap.Set(constant.GROUP_KEY, refconfig.Group)
+	urlMap.Set(constant.VERSION_KEY, refconfig.Version)
+	//getty invoke async or sync
+	urlMap.Set(constant.ASYNC_KEY, strconv.FormatBool(refconfig.async))
+
+	//application info
+	urlMap.Set(constant.APPLICATION_KEY, consumerConfig.ApplicationConfig.Name)
+	urlMap.Set(constant.ORGANIZATION_KEY, consumerConfig.ApplicationConfig.Organization)
+	urlMap.Set(constant.NAME_KEY, consumerConfig.ApplicationConfig.Name)
+	urlMap.Set(constant.MODULE_KEY, consumerConfig.ApplicationConfig.Module)
+	urlMap.Set(constant.APP_VERSION_KEY, consumerConfig.ApplicationConfig.Version)
+	urlMap.Set(constant.OWNER_KEY, consumerConfig.ApplicationConfig.Owner)
+	urlMap.Set(constant.ENVIRONMENT_KEY, consumerConfig.ApplicationConfig.Environment)
+
+	for _, v := range refconfig.Methods {
+		urlMap.Set("methods."+v.Name+"."+constant.LOADBALANCE_KEY, v.Loadbalance)
+		urlMap.Set("methods."+v.Name+"."+constant.RETRIES_KEY, strconv.FormatInt(v.Retries, 10))
+	}
+
+	return urlMap
+
+}
diff --git a/config/support/registry_config.go b/config/support/registry_config.go
new file mode 100644
index 0000000000000000000000000000000000000000..d77e1a8c39499d59f2b181f6b837ede132ab2e18
--- /dev/null
+++ b/config/support/registry_config.go
@@ -0,0 +1,55 @@
+package support
+
+import (
+	"context"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"net/url"
+	"strconv"
+)
+import (
+	log "github.com/AlexStocks/log4go"
+)
+import "github.com/dubbo/go-for-apache-dubbo/config"
+
+type RegistryConfig struct {
+	Id         string `required:"true" yaml:"id"  json:"id,omitempty"`
+	Type       string `required:"true" yaml:"type"  json:"type,omitempty"`
+	TimeoutStr string `yaml:"timeout" default:"5s" json:"timeout,omitempty"` // unit: second
+	Group      string `yaml:"group" json:"group,omitempty"`
+	//for registry
+	Address  string `yaml:"address" json:"address,omitempty"`
+	Username string `yaml:"username" json:"address,omitempty"`
+	Password string `yaml:"password" json:"address,omitempty"`
+}
+
+func loadRegistries(registriesIds []ConfigRegistry, registries []RegistryConfig, roleType config.RoleType) []*config.URL {
+	var urls []*config.URL
+	for _, registry := range registriesIds {
+		for _, registryConf := range registries {
+			if string(registry) == registryConf.Id {
+
+				url, err := config.NewURL(context.TODO(), constant.REGISTRY_PROTOCOL+"://"+registryConf.Address, config.WithParams(registryConf.getUrlMap(roleType)),
+					config.WithUsername(registryConf.Username), config.WithPassword(registryConf.Password),
+				)
+
+				if err != nil {
+					log.Error("The registry id:%s url is invalid ,and will skip the registry, error: %#v", registryConf.Id, err)
+				} else {
+					urls = append(urls, &url)
+				}
+
+			}
+		}
+
+	}
+	return urls
+}
+
+func (regconfig *RegistryConfig) getUrlMap(roleType config.RoleType) url.Values {
+	urlMap := url.Values{}
+	urlMap.Set(constant.GROUP_KEY, regconfig.Group)
+	urlMap.Set(constant.ROLE_KEY, strconv.Itoa(int(roleType)))
+	urlMap.Set(constant.REGISTRY_KEY, regconfig.Type)
+	urlMap.Set(constant.REGISTRY_TIMEOUT_KEY, regconfig.TimeoutStr)
+	return urlMap
+}
diff --git a/config/support/service.go b/config/support/service.go
new file mode 100644
index 0000000000000000000000000000000000000000..1e4a9d4a1addc77d7bed9ab28d0b0cba2ce2604e
--- /dev/null
+++ b/config/support/service.go
@@ -0,0 +1,26 @@
+package support
+
+import "github.com/dubbo/go-for-apache-dubbo/config"
+
+var (
+	conServices = map[string]config.RPCService{} // service name -> service
+	proServices = map[string]config.RPCService{} // service name -> service
+)
+
+// SetConService is called by init() of implement of RPCService
+func SetConService(service config.RPCService) {
+	conServices[service.Service()] = service
+}
+
+// SetProService is called by init() of implement of RPCService
+func SetProService(service config.RPCService) {
+	proServices[service.Service()] = service
+}
+
+func GetConService(name string) config.RPCService {
+	return conServices[name]
+}
+
+func GetProService(name string) config.RPCService {
+	return proServices[name]
+}
diff --git a/config/support/service_config.go b/config/support/service_config.go
new file mode 100644
index 0000000000000000000000000000000000000000..b5e9f16c48603cc896288d4ea334ca0c44fd7eae
--- /dev/null
+++ b/config/support/service_config.go
@@ -0,0 +1,143 @@
+package support
+
+import (
+	"context"
+	"net/url"
+	"strconv"
+	"strings"
+	"sync"
+	"time"
+)
+import (
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+	"go.uber.org/atomic"
+)
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type ServiceConfig struct {
+	context       context.Context
+	Protocol      string           `required:"true"  yaml:"protocol"  json:"protocol,omitempty"` //multi protocol support, split by ','
+	InterfaceName string           `required:"true"  yaml:"interface"  json:"interface,omitempty"`
+	Registries    []ConfigRegistry `required:"true"  yaml:"registries"  json:"registries,omitempty"`
+	Cluster       string           `default:"failover" yaml:"cluster"  json:"cluster,omitempty"`
+	Loadbalance   string           `default:"random" yaml:"loadbalance"  json:"loadbalance,omitempty"`
+	Group         string           `yaml:"group"  json:"group,omitempty"`
+	Version       string           `yaml:"version"  json:"version,omitempty"`
+	Methods       []struct {
+		Name        string `yaml:"name"  json:"name,omitempty"`
+		Retries     int64  `yaml:"retries"  json:"retries,omitempty"`
+		Loadbalance string `yaml:"loadbalance"  json:"loadbalance,omitempty"`
+		Weight      int64  `yaml:"weight"  json:"weight,omitempty"`
+	} `yaml:"methods"  json:"methods,omitempty"`
+	Warmup        string `yaml:"warmup"  json:"warmup,omitempty"`
+	Retries       int64  `yaml:"retries"  json:"retries,omitempty"`
+	unexported    *atomic.Bool
+	exported      *atomic.Bool
+	rpcService    config.RPCService
+	exporters     []protocol.Exporter
+	cacheProtocol protocol.Protocol
+	cacheMutex    sync.Mutex
+}
+
+func NewServiceConfig() *ServiceConfig {
+	return &ServiceConfig{
+		unexported: atomic.NewBool(false),
+		exported:   atomic.NewBool(false),
+	}
+
+}
+
+func (srvconfig *ServiceConfig) Export() error {
+	//TODO: config center start here
+
+	//TODO:delay export
+	if srvconfig.unexported != nil && srvconfig.unexported.Load() {
+		err := jerrors.Errorf("The service %v has already unexported! ", srvconfig.InterfaceName)
+		log.Error(err.Error())
+		return err
+	}
+	if srvconfig.unexported != nil && srvconfig.exported.Load() {
+		log.Warn("The service %v has already exported! ", srvconfig.InterfaceName)
+		return nil
+	}
+
+	regUrls := loadRegistries(srvconfig.Registries, providerConfig.Registries, config.PROVIDER)
+	urlMap := srvconfig.getUrlMap()
+
+	for _, proto := range loadProtocol(srvconfig.Protocol, providerConfig.Protocols) {
+		//registry the service reflect
+		methods, err := config.ServiceMap.Register(proto.Name, srvconfig.rpcService)
+		if err != nil {
+			err := jerrors.Errorf("The service %v  export the protocol %v error! Error message is %v .", srvconfig.InterfaceName, proto.Name, err.Error())
+			log.Error(err.Error())
+			return err
+		}
+		//contextPath := proto.ContextPath
+		//if contextPath == "" {
+		//	contextPath = providerConfig.Path
+		//}
+		url := config.NewURLWithOptions(srvconfig.InterfaceName,
+			config.WithProtocol(proto.Name),
+			config.WithIp(proto.Ip),
+			config.WithPort(proto.Port),
+			config.WithParams(urlMap),
+			config.WithMethods(strings.Split(methods, ",")))
+
+		for _, regUrl := range regUrls {
+			regUrl.SubURL = url
+			invoker := protocol.NewBaseInvoker(*regUrl)
+			srvconfig.cacheMutex.Lock()
+			if srvconfig.cacheProtocol == nil {
+				log.Info("First load the registry protocol!")
+				srvconfig.cacheProtocol = extension.GetProtocolExtension("registry")
+			}
+			srvconfig.cacheMutex.Unlock()
+			exporter := srvconfig.cacheProtocol.Export(invoker)
+			if exporter == nil {
+				panic(jerrors.New("New exporter error"))
+			}
+			srvconfig.exporters = append(srvconfig.exporters, exporter)
+		}
+	}
+	return nil
+
+}
+
+func (srvconfig *ServiceConfig) Implement(s config.RPCService) {
+	srvconfig.rpcService = s
+}
+
+func (srvconfig *ServiceConfig) getUrlMap() url.Values {
+	urlMap := url.Values{}
+	urlMap.Set(constant.INTERFACE_KEY, srvconfig.InterfaceName)
+	urlMap.Set(constant.TIMESTAMP_KEY, strconv.FormatInt(time.Now().Unix(), 10))
+	urlMap.Set(constant.CLUSTER_KEY, srvconfig.Cluster)
+	urlMap.Set(constant.LOADBALANCE_KEY, srvconfig.Loadbalance)
+	urlMap.Set(constant.WARMUP_KEY, srvconfig.Warmup)
+	urlMap.Set(constant.RETRIES_KEY, strconv.FormatInt(srvconfig.Retries, 10))
+	urlMap.Set(constant.GROUP_KEY, srvconfig.Group)
+	urlMap.Set(constant.VERSION_KEY, srvconfig.Version)
+	//application info
+	urlMap.Set(constant.APPLICATION_KEY, providerConfig.ApplicationConfig.Name)
+	urlMap.Set(constant.ORGANIZATION_KEY, providerConfig.ApplicationConfig.Organization)
+	urlMap.Set(constant.NAME_KEY, providerConfig.ApplicationConfig.Name)
+	urlMap.Set(constant.MODULE_KEY, providerConfig.ApplicationConfig.Module)
+	urlMap.Set(constant.APP_VERSION_KEY, providerConfig.ApplicationConfig.Version)
+	urlMap.Set(constant.OWNER_KEY, providerConfig.ApplicationConfig.Owner)
+	urlMap.Set(constant.ENVIRONMENT_KEY, providerConfig.ApplicationConfig.Environment)
+
+	for _, v := range srvconfig.Methods {
+		urlMap.Set("methods."+v.Name+"."+constant.LOADBALANCE_KEY, v.Loadbalance)
+		urlMap.Set("methods."+v.Name+"."+constant.RETRIES_KEY, strconv.FormatInt(v.Retries, 10))
+		urlMap.Set("methods."+v.Name+"."+constant.WEIGHT_KEY, strconv.FormatInt(v.Weight, 10))
+	}
+
+	return urlMap
+
+}
diff --git a/config/url.go b/config/url.go
new file mode 100644
index 0000000000000000000000000000000000000000..a43337f51b0a559d76195bffe37539ebc356e4b3
--- /dev/null
+++ b/config/url.go
@@ -0,0 +1,268 @@
+package config
+
+import (
+	"context"
+	"fmt"
+	"net"
+	"net/url"
+	"strconv"
+	"strings"
+)
+
+import (
+	jerrors "github.com/juju/errors"
+)
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+)
+
+/////////////////////////////////
+// dubbo role type
+/////////////////////////////////
+
+const (
+	CONSUMER = iota
+	CONFIGURATOR
+	ROUTER
+	PROVIDER
+)
+
+var (
+	DubboNodes = [...]string{"consumers", "configurators", "routers", "providers"}
+	DubboRole  = [...]string{"consumer", "", "", "provider"}
+)
+
+type RoleType int
+
+func (t RoleType) String() string {
+	return DubboNodes[t]
+}
+
+func (t RoleType) Role() string {
+	return DubboRole[t]
+}
+
+type baseUrl struct {
+	Protocol     string
+	Location     string // ip+port
+	Ip           string
+	Port         string
+	Params       url.Values
+	PrimitiveURL string
+	ctx          context.Context
+}
+
+type URL struct {
+	baseUrl
+	Path     string // like  /com.ikurento.dubbo.UserProvider3
+	Username string
+	Password string
+	Methods  []string
+	//special for registry
+	SubURL *URL
+}
+
+type option func(*URL)
+
+func WithUsername(username string) option {
+	return func(url *URL) {
+		url.Username = username
+	}
+}
+
+func WithPassword(pwd string) option {
+	return func(url *URL) {
+		url.Password = pwd
+	}
+}
+
+func WithMethods(methods []string) option {
+	return func(url *URL) {
+		url.Methods = methods
+	}
+}
+
+func WithParams(params url.Values) option {
+	return func(url *URL) {
+		url.Params = params
+	}
+}
+
+func WithProtocol(proto string) option {
+	return func(url *URL) {
+		url.Protocol = proto
+	}
+}
+func WithIp(ip string) option {
+	return func(url *URL) {
+		url.Ip = ip
+	}
+}
+
+func WithPort(port string) option {
+	return func(url *URL) {
+		url.Port = port
+	}
+}
+
+//func WithPath(path string) option {
+//	return func(url *URL) {
+//		url.Path = path
+//	}
+//}
+
+func NewURLWithOptions(service string, opts ...option) *URL {
+	url := &URL{
+		Path: "/" + service,
+	}
+	for _, opt := range opts {
+		opt(url)
+	}
+	url.Location = url.Ip + ":" + url.Port
+	return url
+}
+
+func NewURL(ctx context.Context, urlString string, opts ...option) (URL, error) {
+
+	var (
+		err          error
+		rawUrlString string
+		serviceUrl   *url.URL
+		s            = URL{baseUrl: baseUrl{ctx: ctx}}
+	)
+
+	// new a null instance
+	if urlString == "" {
+		return s, nil
+	}
+
+	rawUrlString, err = url.QueryUnescape(urlString)
+	if err != nil {
+		return s, jerrors.Errorf("url.QueryUnescape(%s),  error{%v}", urlString, err)
+	}
+
+	//rawUrlString = "//" + rawUrlString
+	serviceUrl, err = url.Parse(rawUrlString)
+	if err != nil {
+		return s, jerrors.Errorf("url.Parse(url string{%s}),  error{%v}", rawUrlString, err)
+	}
+
+	s.Params, err = url.ParseQuery(serviceUrl.RawQuery)
+	if err != nil {
+		return s, jerrors.Errorf("url.ParseQuery(raw url string{%s}),  error{%v}", serviceUrl.RawQuery, err)
+	}
+
+	s.PrimitiveURL = urlString
+	s.Protocol = serviceUrl.Scheme
+	s.Username = serviceUrl.User.Username()
+	s.Password, _ = serviceUrl.User.Password()
+	s.Location = serviceUrl.Host
+	s.Path = serviceUrl.Path
+	if strings.Contains(s.Location, ":") {
+		s.Ip, s.Port, err = net.SplitHostPort(s.Location)
+		if err != nil {
+			return s, jerrors.Errorf("net.SplitHostPort(Url.Host{%s}), error{%v}", s.Location, err)
+		}
+	}
+	//
+	//timeoutStr := s.Params.Get("timeout")
+	//if len(timeoutStr) == 0 {
+	//	timeoutStr = s.Params.Get("default.timeout")
+	//}
+	//if len(timeoutStr) != 0 {
+	//	timeout, err := strconv.Atoi(timeoutStr)
+	//	if err == nil && timeout != 0 {
+	//		s.Timeout = time.Duration(timeout * 1e6) // timeout unit is millisecond
+	//	}
+	//}
+	for _, opt := range opts {
+		opt(&s)
+	}
+	//fmt.Println(s.String())
+	return s, nil
+}
+
+//
+//func (c URL) Key() string {
+//	return fmt.Sprintf(
+//		"%s://%s:%s@%s:%s/%s",
+//		c.Protocol, c.Username, c.Password, c.Ip, c.Port, c.Path)
+//}
+
+func (c URL) URLEqual(url URL) bool {
+	c.Ip = ""
+	c.Port = ""
+	url.Ip = ""
+	url.Port = ""
+	if c.Key() != url.Key() {
+		return false
+	}
+	return true
+}
+
+//func (c SubURL) String() string {
+//	return fmt.Sprintf(
+//		"DefaultServiceURL{protocol:%s, Location:%s, Path:%s, Ip:%s, Port:%s, "+
+//			"Timeout:%s, Version:%s, Group:%s,  Params:%+v}",
+//		c.protocol, c.Location, c.Path, c.Ip, c.Port,
+//		c.Timeout, c.Version, c.Group, c.Params)
+//}
+
+func (c URL) String() string {
+	buildString := fmt.Sprintf(
+		"%s://%s:%s@%s:%s%s?",
+		c.Protocol, c.Username, c.Password, c.Ip, c.Port, c.Path)
+	for k, v := range c.Params {
+		buildString += "&" + k + "=" + v[0]
+	}
+	return buildString
+}
+
+func (c URL) Key() string {
+	buildString := fmt.Sprintf(
+		"%s://%s:%s@%s:%s/%s?group=%s&version=%s",
+		c.Protocol, c.Username, c.Password, c.Ip, c.Port, c.GetParam(constant.INTERFACE_KEY, strings.TrimPrefix(c.Path, "/")), c.GetParam(constant.GROUP_KEY, ""), c.GetParam(constant.VERSION_KEY, constant.DEFAULT_VERSION))
+
+	return buildString
+}
+
+func (c URL) Context() context.Context {
+	return c.ctx
+}
+
+func (c URL) Service() string {
+	return strings.TrimPrefix(c.Path, "/")
+}
+func (c URL) GetParam(s string, d string) string {
+	var r string
+	if r = c.Params.Get(s); r == "" {
+		r = d
+	}
+	return r
+}
+
+func (c URL) GetParamInt(s string, d int64) int64 {
+	var r int
+	var err error
+	if r, err = strconv.Atoi(c.Params.Get(s)); r == 0 || err != nil {
+		return d
+	}
+	return int64(r)
+}
+
+func (c URL) GetMethodParamInt(method string, key string, d int64) int64 {
+	var r int
+	var err error
+	if r, err = strconv.Atoi(c.Params.Get("methods." + method + "." + key)); r == 0 || err != nil {
+		return d
+	}
+	return int64(r)
+}
+
+func (c URL) GetMethodParam(method string, key string, d string) string {
+	var r string
+	if r = c.Params.Get(c.Params.Get("methods." + method + "." + key)); r == "" {
+		r = d
+	}
+	return r
+}
diff --git a/dubbo/rpc.go b/dubbo/rpc.go
deleted file mode 100644
index f20fc9603eeb630a525ee8a87e638eec95aead96..0000000000000000000000000000000000000000
--- a/dubbo/rpc.go
+++ /dev/null
@@ -1,119 +0,0 @@
-package dubbo
-
-import (
-	"reflect"
-	"sync"
-	"unicode"
-	"unicode/utf8"
-)
-
-import (
-	log "github.com/AlexStocks/log4go"
-)
-
-var (
-	typeOfError = reflect.TypeOf((*error)(nil)).Elem()
-)
-
-type GettyRPCService interface {
-	Service() string // Service Interface
-	Version() string
-}
-
-type methodType struct {
-	sync.Mutex
-	method    reflect.Method
-	CtxType   reflect.Type // type of the request context
-	ArgType   reflect.Type
-	ReplyType reflect.Type
-}
-
-type service struct {
-	name     string
-	rcvr     reflect.Value
-	rcvrType reflect.Type
-	method   map[string]*methodType
-}
-
-// Is this an exported - upper case - name
-func isExported(name string) bool {
-	rune, _ := utf8.DecodeRuneInString(name)
-	return unicode.IsUpper(rune)
-}
-
-// Is this type exported or a builtin?
-func isExportedOrBuiltinType(t reflect.Type) bool {
-	for t.Kind() == reflect.Ptr {
-		t = t.Elem()
-	}
-	// PkgPath will be non-empty even for an exported type,
-	// so we need to check the type name as well.
-	return isExported(t.Name()) || t.PkgPath() == ""
-}
-
-// suitableMethods returns suitable Rpc methods of typ
-func suitableMethods(typ reflect.Type) (string, map[string]*methodType) {
-	methods := make(map[string]*methodType)
-	mts := ""
-	for m := 0; m < typ.NumMethod(); m++ {
-		method := typ.Method(m)
-		if mt := suiteMethod(method); mt != nil {
-			methods[method.Name] = mt
-			mts += method.Name + ","
-		}
-	}
-	return mts, methods
-}
-
-// suiteMethod returns a suitable Rpc methodType
-func suiteMethod(method reflect.Method) *methodType {
-	mtype := method.Type
-	mname := method.Name
-
-	// Method must be exported.
-	if method.PkgPath != "" {
-		return nil
-	}
-
-	var replyType, argType, ctxType reflect.Type
-	switch mtype.NumIn() {
-	case 3:
-		argType = mtype.In(1)
-		replyType = mtype.In(2)
-	case 4:
-		ctxType = mtype.In(1)
-		argType = mtype.In(2)
-		replyType = mtype.In(3)
-	default:
-		log.Error("method %s of mtype %v has wrong number of in parameters %d; needs exactly 3/4",
-			mname, mtype, mtype.NumIn())
-		return nil
-	}
-	// First arg need not be a pointer.
-	if !isExportedOrBuiltinType(argType) {
-		log.Error("argument type of method %q is not exported %v", mname, argType)
-		return nil
-	}
-	// Second arg must be a pointer.
-	if replyType.Kind() != reflect.Ptr {
-		log.Error("reply type of method %q is not a pointer %v", mname, replyType)
-		return nil
-	}
-	// Reply type must be exported.
-	if !isExportedOrBuiltinType(replyType) {
-		log.Error("reply type of method %s not exported{%v}", mname, replyType)
-		return nil
-	}
-	// Method needs one out.
-	if mtype.NumOut() != 1 {
-		log.Error("method %q has %d out parameters; needs exactly 1", mname, mtype.NumOut())
-		return nil
-	}
-	// The return type of the method must be error.
-	if returnType := mtype.Out(0); returnType != typeOfError {
-		log.Error("return type %s of method %q is not error", returnType, mname)
-		return nil
-	}
-
-	return &methodType{method: method, ArgType: argType, ReplyType: replyType, CtxType: ctxType}
-}
diff --git a/dubbo/server.go b/dubbo/server.go
deleted file mode 100644
index 4766dbe2f61fe20343aa6b9175f719f8afd39c30..0000000000000000000000000000000000000000
--- a/dubbo/server.go
+++ /dev/null
@@ -1,252 +0,0 @@
-package dubbo
-
-import (
-	"fmt"
-	"github.com/dubbo/dubbo-go/plugins"
-	"net"
-	"reflect"
-	"strconv"
-)
-
-import (
-	"github.com/AlexStocks/getty"
-	"github.com/AlexStocks/goext/net"
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-type Option func(*Options)
-
-type Options struct {
-	Registry        registry.Registry
-	ConfList        []ServerConfig
-	ServiceConfList []registry.ServiceConfig
-}
-
-func newOptions(opt ...Option) Options {
-	opts := Options{}
-	for _, o := range opt {
-		o(&opts)
-	}
-
-	if opts.Registry == nil {
-		panic("server.Options.Registry is nil")
-	}
-
-	return opts
-}
-
-// Registry used for discovery
-func Registry(r registry.Registry) Option {
-	return func(o *Options) {
-		o.Registry = r
-	}
-}
-
-func ConfList(confList []ServerConfig) Option {
-	return func(o *Options) {
-		o.ConfList = confList
-		for i := 0; i < len(o.ConfList); i++ {
-			if err := o.ConfList[i].CheckValidity(); err != nil {
-				log.Error("ServerConfig check failed: ", err)
-				o.ConfList = []ServerConfig{}
-				return
-			}
-			if o.ConfList[i].IP == "" {
-				o.ConfList[i].IP, _ = gxnet.GetLocalIP()
-			}
-		}
-	}
-}
-
-func ServiceConfList(confList []registry.ServiceConfig) Option {
-	return func(o *Options) {
-		o.ServiceConfList = confList
-		if o.ServiceConfList == nil {
-			o.ServiceConfList = []registry.ServiceConfig{}
-		}
-	}
-}
-
-type serviceMap map[string]*service
-
-type Server struct {
-	opts            Options
-	indexOfConfList int
-	srvs            []serviceMap
-	tcpServerList   []getty.Server
-}
-
-func NewServer(opts ...Option) *Server {
-	options := newOptions(opts...)
-	num := len(options.ConfList)
-	servers := make([]serviceMap, len(options.ConfList))
-
-	for i := 0; i < num; i++ {
-		servers[i] = map[string]*service{}
-	}
-
-	s := &Server{
-		opts: options,
-		srvs: servers,
-	}
-
-	return s
-}
-
-// Register export services and register with the registry
-func (s *Server) Register(rcvr GettyRPCService) error {
-
-	serviceConf := plugins.DefaultProviderServiceConfig()()
-
-	opts := s.opts
-
-	serviceConf.SetService(rcvr.Service())
-	serviceConf.SetVersion(rcvr.Version())
-
-	flag := false
-	serviceNum := len(opts.ServiceConfList)
-	serverNum := len(opts.ConfList)
-	for i := 0; i < serviceNum; i++ {
-		if opts.ServiceConfList[i].Service() == serviceConf.Service() &&
-			opts.ServiceConfList[i].Version() == serviceConf.Version() {
-
-			serviceConf.SetProtocol(opts.ServiceConfList[i].Protocol())
-			serviceConf.SetGroup(opts.ServiceConfList[i].Group())
-
-			for j := 0; j < serverNum; j++ {
-				if opts.ConfList[j].Protocol == serviceConf.Protocol() {
-					rcvrName := reflect.Indirect(reflect.ValueOf(rcvr)).Type().Name()
-					svc := &service{
-						rcvrType: reflect.TypeOf(rcvr),
-						rcvr:     reflect.ValueOf(rcvr),
-					}
-					if rcvrName == "" {
-						s := "rpc.Register: no service name for type " + svc.rcvrType.String()
-						log.Error(s)
-						return jerrors.New(s)
-					}
-					if !isExported(rcvrName) {
-						s := "rpc.Register: type " + rcvrName + " is not exported"
-						log.Error(s)
-						return jerrors.New(s)
-					}
-
-					svc.name = rcvr.Service() // service name is from 'Service()'
-					if _, present := s.srvs[j][svc.name]; present {
-						return jerrors.New("rpc: service already defined: " + svc.name)
-					}
-
-					// Install the methods
-					mts, methods := suitableMethods(svc.rcvrType)
-					svc.method = methods
-
-					if len(svc.method) == 0 {
-						// To help the user, see if a pointer receiver would work.
-						mts, methods = suitableMethods(reflect.PtrTo(svc.rcvrType))
-						str := "rpc.Register: type " + rcvrName + " has no exported methods of suitable type"
-						if len(methods) != 0 {
-							str = "rpc.Register: type " + rcvrName + " has no exported methods of suitable type (" +
-								"hint: pass a pointer to value of that type)"
-						}
-						log.Error(str)
-
-						return jerrors.New(str)
-					}
-
-					s.srvs[j][svc.name] = svc
-
-					serviceConf.SetMethods(mts)
-					serviceConf.SetPath(opts.ConfList[j].Address())
-
-					err := opts.Registry.Register(serviceConf)
-					if err != nil {
-						return err
-					}
-					flag = true
-				}
-			}
-		}
-	}
-
-	if !flag {
-		return jerrors.Errorf("fail to register Handler{service:%s, version:%s}",
-			serviceConf.Service, serviceConf.Version)
-	}
-	return nil
-}
-
-func (s *Server) newSession(session getty.Session) error {
-	var (
-		ok      bool
-		tcpConn *net.TCPConn
-	)
-	conf := s.opts.ConfList[s.indexOfConfList]
-
-	if conf.GettySessionParam.CompressEncoding {
-		session.SetCompressType(getty.CompressZip)
-	}
-
-	if tcpConn, ok = session.Conn().(*net.TCPConn); !ok {
-		panic(fmt.Sprintf("%s, session.conn{%#v} is not tcp connection\n", session.Stat(), session.Conn()))
-	}
-
-	tcpConn.SetNoDelay(conf.GettySessionParam.TcpNoDelay)
-	tcpConn.SetKeepAlive(conf.GettySessionParam.TcpKeepAlive)
-	if conf.GettySessionParam.TcpKeepAlive {
-		tcpConn.SetKeepAlivePeriod(conf.GettySessionParam.keepAlivePeriod)
-	}
-	tcpConn.SetReadBuffer(conf.GettySessionParam.TcpRBufSize)
-	tcpConn.SetWriteBuffer(conf.GettySessionParam.TcpWBufSize)
-
-	session.SetName(conf.GettySessionParam.SessionName)
-	session.SetMaxMsgLen(conf.GettySessionParam.MaxMsgLen)
-	session.SetPkgHandler(NewRpcServerPackageHandler(s, s.srvs[s.indexOfConfList]))
-	session.SetEventListener(NewRpcServerHandler(conf.SessionNumber, conf.sessionTimeout))
-	session.SetRQLen(conf.GettySessionParam.PkgRQSize)
-	session.SetWQLen(conf.GettySessionParam.PkgWQSize)
-	session.SetReadTimeout(conf.GettySessionParam.tcpReadTimeout)
-	session.SetWriteTimeout(conf.GettySessionParam.tcpWriteTimeout)
-	session.SetCronPeriod((int)(conf.sessionTimeout.Nanoseconds() / 1e6))
-	session.SetWaitTime(conf.GettySessionParam.waitTimeout)
-	log.Debug("app accepts new session:%s\n", session.Stat())
-
-	return nil
-}
-
-func (s *Server) Start() {
-	var (
-		addr      string
-		tcpServer getty.Server
-	)
-
-	if len(s.opts.ConfList) == 0 {
-		panic("ConfList is nil")
-	}
-
-	for i := 0; i < len(s.opts.ConfList); i++ {
-		addr = gxnet.HostAddress2(s.opts.ConfList[i].IP, strconv.Itoa(s.opts.ConfList[i].Port))
-		tcpServer = getty.NewTCPServer(
-			getty.WithLocalAddress(addr),
-		)
-		s.indexOfConfList = i
-		tcpServer.RunEventLoop(s.newSession)
-		log.Debug("s bind addr{%s} ok!", addr)
-		s.tcpServerList = append(s.tcpServerList, tcpServer)
-	}
-
-}
-
-func (s *Server) Stop() {
-	list := s.tcpServerList
-	s.tcpServerList = nil
-	if list != nil {
-		for _, tcpServer := range list {
-			tcpServer.Close()
-		}
-	}
-}
diff --git a/examples/client_config.go b/examples/client_config.go
deleted file mode 100644
index bf871ee7fbc7fb1ace9905d20950fe57ff098a5d..0000000000000000000000000000000000000000
--- a/examples/client_config.go
+++ /dev/null
@@ -1,121 +0,0 @@
-package examples
-
-import (
-	"fmt"
-	"io/ioutil"
-	"os"
-	"path"
-	"time"
-)
-
-import (
-	"github.com/AlexStocks/goext/log"
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-	"gopkg.in/yaml.v2"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
-)
-
-const (
-	APP_CONF_FILE     = "APP_CONF_FILE"
-	APP_LOG_CONF_FILE = "APP_LOG_CONF_FILE"
-)
-
-type (
-	// Client holds supported types by the multiconfig package
-	ClientConfig struct {
-		// pprof
-		Pprof_Enabled bool `default:"false" yaml:"pprof_enabled" json:"pprof_enabled,omitempty"`
-		Pprof_Port    int  `default:"10086"  yaml:"pprof_port" json:"pprof_port,omitempty"`
-
-		// client
-		Connect_Timeout string `default:"100ms"  yaml:"connect_timeout" json:"connect_timeout,omitempty"`
-		ConnectTimeout  time.Duration
-
-		Request_Timeout string `yaml:"request_timeout" default:"5s" json:"request_timeout,omitempty"` // 500ms, 1m
-		RequestTimeout  time.Duration
-
-		// codec & selector & transport & registry
-		Selector     string `default:"cache"  yaml:"selector" json:"selector,omitempty"`
-		Selector_TTL string `default:"10m"  yaml:"selector_ttl" json:"selector_ttl,omitempty"`
-		//client load balance algorithm
-		ClientLoadBalance string `default:"round_robin"  yaml:"client_load_balance" json:"client_load_balance,omitempty"`
-		Registry          string `default:"zookeeper"  yaml:"registry" json:"registry,omitempty"`
-		// application
-		Application_Config registry.ApplicationConfig `yaml:"application_config" json:"application_config,omitempty"`
-		ZkRegistryConfig   zookeeper.ZkRegistryConfig `yaml:"zk_registry_config" json:"zk_registry_config,omitempty"`
-		// 一个客户端只允许使用一个service的其中一个group和其中一个version
-		ServiceConfigType    string                   `default:"default" yaml:"service_config_type" json:"service_config_type,omitempty"`
-		ServiceConfigList    []registry.ServiceConfig `yaml:"-"`
-		ServiceConfigMapList []map[string]string      `yaml:"service_list" json:"service_list,omitempty"`
-	}
-)
-
-func InitClientConfig() *ClientConfig {
-
-	var (
-		clientConfig *ClientConfig
-		confFile     string
-	)
-
-	// configure
-	confFile = os.Getenv(APP_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("application configure file name is nil"))
-		return nil // I know it is of no usage. Just Err Protection.
-	}
-	if path.Ext(confFile) != ".yml" {
-		panic(fmt.Sprintf("application configure file name{%v} suffix must be .yml", confFile))
-		return nil
-	}
-	clientConfig = new(ClientConfig)
-
-	confFileStream, err := ioutil.ReadFile(confFile)
-	if err != nil {
-		panic(fmt.Sprintf("ioutil.ReadFile(file:%s) = error:%s", confFile, jerrors.ErrorStack(err)))
-		return nil
-	}
-	err = yaml.Unmarshal(confFileStream, clientConfig)
-	if err != nil {
-		panic(fmt.Sprintf("yaml.Unmarshal() = error:%s", jerrors.ErrorStack(err)))
-		return nil
-	}
-
-	//动态加载service config
-	//设置默认ProviderServiceConfig类
-	plugins.SetDefaultServiceConfig(clientConfig.ServiceConfigType)
-
-	for _, service := range clientConfig.ServiceConfigMapList {
-		svc := plugins.DefaultServiceConfig()()
-		svc.SetProtocol(service["protocol"])
-		svc.SetService(service["service"])
-		clientConfig.ServiceConfigList = append(clientConfig.ServiceConfigList, svc)
-	}
-	//动态加载service config  end
-
-	if clientConfig.ZkRegistryConfig.Timeout, err = time.ParseDuration(clientConfig.ZkRegistryConfig.TimeoutStr); err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Registry_Config.Timeout:%#v) = error:%s", clientConfig.ZkRegistryConfig.TimeoutStr, err))
-		return nil
-	}
-
-	gxlog.CInfo("config{%#v}\n", clientConfig)
-
-	// log
-	confFile = os.Getenv(APP_LOG_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("log configure file name is nil"))
-		return nil
-	}
-	if path.Ext(confFile) != ".xml" {
-		panic(fmt.Sprintf("log configure file name{%v} suffix must be .xml", confFile))
-		return nil
-	}
-	log.LoadConfiguration(confFile)
-
-	return clientConfig
-}
diff --git a/examples/dubbo/go-client/app/client.go b/examples/dubbo/go-client/app/client.go
index 0cd19e411706d4810b4cc3e31a1f8d5d530a7118..4ea5f5c38c0d205476b0bc14452ed99934522524 100644
--- a/examples/dubbo/go-client/app/client.go
+++ b/examples/dubbo/go-client/app/client.go
@@ -1,10 +1,9 @@
 package main
 
 import (
+	"context"
 	"fmt"
-	"github.com/dubbo/dubbo-go/dubbo"
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
+	"github.com/dubbogo/hessian2"
 	"net/http"
 	_ "net/http/pprof"
 	"os"
@@ -18,132 +17,62 @@ import (
 	"github.com/AlexStocks/goext/log"
 	"github.com/AlexStocks/goext/net"
 	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/client/invoker"
-	"github.com/dubbo/dubbo-go/examples"
-	"github.com/dubbo/dubbo-go/public"
-	"github.com/dubbo/dubbo-go/registry"
+	_ "github.com/dubbo/go-for-apache-dubbo/protocol/dubbo"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/protocol"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/filter/imp"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/loadbalance"
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/support"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/zookeeper"
+
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
 )
 
 var (
 	survivalTimeout int = 10e9
-	clientInvoker   *invoker.Invoker
 )
 
+// they are necessary:
+// 		export CONF_CONSUMER_FILE_PATH="xxx"
+// 		export APP_LOG_CONF_FILE="xxx"
 func main() {
 
-	clientConfig := examples.InitClientConfig()
-	initProfiling(clientConfig)
-	initClient(clientConfig)
+	hessian.RegisterJavaEnum(Gender(MAN))
+	hessian.RegisterJavaEnum(Gender(WOMAN))
+	hessian.RegisterPOJO(&User{})
 
-	time.Sleep(3e9)
+	conMap, _ := support.Load()
+	if conMap == nil {
+		panic("conMap is nil")
+	}
 
-	gxlog.CInfo("\n\n\nstart to test dubbo")
-	testDubborpc(clientConfig, "A003")
+	initProfiling()
 
 	time.Sleep(3e9)
 
-	initSignal()
-}
-
-func initClient(clientConfig *examples.ClientConfig) {
-	var (
-		err       error
-		codecType public.CodecType
-	)
-
-	if clientConfig == nil {
-		panic(fmt.Sprintf("clientConfig is nil"))
-		return
-	}
-
-	// registry
-	clientRegistry, err := plugins.PluggableRegistries[clientConfig.Registry](
-		registry.WithDubboType(registry.CONSUMER),
-		registry.WithApplicationConf(clientConfig.Application_Config),
-		zookeeper.WithRegistryConf(clientConfig.ZkRegistryConfig),
-	)
-	if err != nil {
-		panic(fmt.Sprintf("fail to init registry.Registy, err:%s", jerrors.ErrorStack(err)))
-		return
-	}
-
-	// consumer
-	clientConfig.RequestTimeout, err = time.ParseDuration(clientConfig.Request_Timeout)
-	if err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Request_Timeout{%#v}) = error{%v}",
-			clientConfig.Request_Timeout, err))
-		return
-	}
-	clientConfig.ConnectTimeout, err = time.ParseDuration(clientConfig.Connect_Timeout)
+	gxlog.CInfo("\n\n\nstart to test dubbo")
+	user := &User{}
+	err := conMap["com.ikurento.user.UserProvider"].GetRPCService().(*UserProvider).GetUser(context.TODO(), []interface{}{"A003"}, user)
 	if err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Connect_Timeout{%#v}) = error{%v}",
-			clientConfig.Connect_Timeout, err))
-		return
+		panic(err)
 	}
+	gxlog.CInfo("response result: %v", user)
 
-	for idx := range clientConfig.ServiceConfigList {
-		codecType = public.GetCodecType(clientConfig.ServiceConfigList[idx].Protocol())
-		if codecType == public.CODECTYPE_UNKNOWN {
-			panic(fmt.Sprintf("unknown protocol %s", clientConfig.ServiceConfigList[idx].Protocol()))
-		}
-	}
-
-	for _, service := range clientConfig.ServiceConfigList {
-		err = clientRegistry.Register(service)
-		if err != nil {
-			panic(fmt.Sprintf("registry.Register(service{%#v}) = error{%v}", service, jerrors.ErrorStack(err)))
-			return
-		}
-	}
-
-	//read the client lb config in config.yml
-	configClientLB := plugins.PluggableLoadbalance[clientConfig.ClientLoadBalance]()
-
-	//init dubbo rpc client & init invoker
-	var cltD *dubbo.Client
-
-	cltD, err = dubbo.NewClient(&dubbo.ClientConfig{
-		PoolSize:        64,
-		PoolTTL:         600,
-		ConnectionNum:   2, // 不能太大
-		FailFastTimeout: "5s",
-		SessionTimeout:  "20s",
-		HeartbeatPeriod: "5s",
-		GettySessionParam: dubbo.GettySessionParam{
-			CompressEncoding: false, // 必须false
-			TcpNoDelay:       true,
-			KeepAlivePeriod:  "120s",
-			TcpRBufSize:      262144,
-			TcpKeepAlive:     true,
-			TcpWBufSize:      65536,
-			PkgRQSize:        1024,
-			PkgWQSize:        512,
-			TcpReadTimeout:   "1s",
-			TcpWriteTimeout:  "5s",
-			WaitTimeout:      "1s",
-			MaxMsgLen:        1024,
-			SessionName:      "client",
-		},
-	})
+	gxlog.CInfo("\n\n\nstart to test dubbo illegal method")
+	err = conMap["com.ikurento.user.UserProvider"].GetRPCService().(*UserProvider).GetUser1(context.TODO(), []interface{}{"A003"}, user)
 	if err != nil {
-		log.Error("hessian.NewClient(conf) = error:%s", jerrors.ErrorStack(err))
-		return
+		panic(err)
 	}
-	clientInvoker, err = invoker.NewInvoker(clientRegistry,
-		invoker.WithDubboClient(cltD),
-		invoker.WithLBSelector(configClientLB))
-}
 
-func uninitClient() {
-	log.Close()
+	initSignal()
 }
 
-func initProfiling(clientConfig *examples.ClientConfig) {
-	if !clientConfig.Pprof_Enabled {
+func initProfiling() {
+	if !support.GetProviderConfig().Pprof_Enabled {
 		return
 	}
 	const (
@@ -159,7 +88,7 @@ func initProfiling(clientConfig *examples.ClientConfig) {
 	if err != nil {
 		panic("cat not get local ip!")
 	}
-	addr = ip + ":" + strconv.Itoa(clientConfig.Pprof_Port)
+	addr = ip + ":" + strconv.Itoa(support.GetProviderConfig().Pprof_Port)
 	log.Info("App Profiling startup on address{%v}", addr+PprofPath)
 
 	go func() {
@@ -177,7 +106,7 @@ func initSignal() {
 		log.Info("get signal %s", sig.String())
 		switch sig {
 		case syscall.SIGHUP:
-		// reload()
+			// reload()
 		default:
 			go time.AfterFunc(time.Duration(survivalTimeout)*time.Second, func() {
 				log.Warn("app exit now by force...")
@@ -185,7 +114,6 @@ func initSignal() {
 			})
 
 			// 要么fastFailTimeout时间内执行完毕下面的逻辑然后程序退出,要么执行上面的超时函数程序强行退出
-			uninitClient()
 			fmt.Println("app exit now...")
 			return
 		}
diff --git a/examples/dubbo/go-client/app/test.go b/examples/dubbo/go-client/app/test.go
deleted file mode 100644
index bea2d95d9af2a46d5e60a2ec540b685e1cc2289d..0000000000000000000000000000000000000000
--- a/examples/dubbo/go-client/app/test.go
+++ /dev/null
@@ -1,59 +0,0 @@
-package main
-
-import (
-	"fmt"
-	_ "net/http/pprof"
-)
-
-import (
-	// "github.com/AlexStocks/goext/log"
-	log "github.com/AlexStocks/log4go"
-	"github.com/dubbogo/hessian2"
-	jerrors "github.com/juju/errors"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/dubbo"
-	"github.com/dubbo/dubbo-go/examples"
-	"github.com/dubbo/dubbo-go/public"
-)
-
-func testDubborpc(clientConfig *examples.ClientConfig, userKey string) {
-	var (
-		err        error
-		svc        string
-		method     string
-		serviceIdx int
-		user       *DubboUser
-	)
-	serviceIdx = -1
-	svc = "com.ikurento.user.UserProvider"
-	for i := range clientConfig.ServiceConfigList {
-		if clientConfig.ServiceConfigList[i].Service() == svc && clientConfig.ServiceConfigList[i].Protocol() == public.CODECTYPE_DUBBO.String() {
-			serviceIdx = i
-			break
-		}
-	}
-	if serviceIdx == -1 {
-		panic(fmt.Sprintf("can not find service in config service list:%#v", clientConfig.ServiceConfigList))
-	}
-
-	// Create request
-	method = string("GetUser")
-
-	// registry pojo
-	hessian.RegisterJavaEnum(Gender(MAN))
-	hessian.RegisterJavaEnum(Gender(WOMAN))
-	hessian.RegisterPOJO(&DubboUser{})
-
-	user = new(DubboUser)
-	defer clientInvoker.DubboClient.Close()
-	err = clientInvoker.DubboCall(1, clientConfig.ServiceConfigList[serviceIdx], method, []interface{}{userKey}, user, dubbo.WithCallRequestTimeout(10e9), dubbo.WithCallResponseTimeout(10e9), dubbo.WithCallSerialID(dubbo.S_Dubbo))
-	// Call service
-	if err != nil {
-		log.Error("client.Call() return error:%+v", jerrors.ErrorStack(err))
-		return
-	}
-
-	log.Info("response result:%s", user)
-}
diff --git a/examples/dubbo/go-client/app/user.go b/examples/dubbo/go-client/app/user.go
index bc2212f532e9e4d349d92df6fe55d2432931b07a..4b805616b2ec3e59951b6309688b47d8271c8240 100644
--- a/examples/dubbo/go-client/app/user.go
+++ b/examples/dubbo/go-client/app/user.go
@@ -1,6 +1,7 @@
 package main
 
 import (
+	"context"
 	"fmt"
 	"strconv"
 	"time"
@@ -10,8 +11,16 @@ import (
 	"github.com/dubbogo/hessian2"
 )
 
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+)
+
 type Gender hessian.JavaEnum
 
+func init() {
+	support.SetConService(new(UserProvider))
+}
+
 const (
 	MAN hessian.JavaEnum = iota
 	WOMAN
@@ -49,7 +58,8 @@ func (g Gender) EnumValue(s string) hessian.JavaEnum {
 	return hessian.InvalidJavaEnum
 }
 
-type DubboUser struct {
+type User struct {
+	// !!! Cannot define lowercase names of variable
 	Id   string
 	Name string
 	Age  int32
@@ -57,13 +67,26 @@ type DubboUser struct {
 	Sex  Gender // 注意此处,java enum Object <--> go string
 }
 
-func (u DubboUser) String() string {
+func (u User) String() string {
 	return fmt.Sprintf(
 		"User{Id:%s, Name:%s, Age:%d, Time:%s, Sex:%s}",
 		u.Id, u.Name, u.Age, u.Time, u.Sex,
 	)
 }
 
-func (DubboUser) JavaClassName() string {
+func (User) JavaClassName() string {
 	return "com.ikurento.user.User"
 }
+
+type UserProvider struct {
+	GetUser  func(ctx context.Context, req []interface{}, rsp *User) error
+	GetUser1 func(ctx context.Context, req []interface{}, rsp *User) error
+}
+
+func (u *UserProvider) Service() string {
+	return "com.ikurento.user.UserProvider"
+}
+
+func (u *UserProvider) Version() string {
+	return ""
+}
diff --git a/examples/dubbo/go-client/assembly/bin/load.sh b/examples/dubbo/go-client/assembly/bin/load.sh
index 72127283082e62d9d2bdf2a4b4934db56674535b..5bba2a26c858dad918ccd6bc60881151d906d25c 100644
--- a/examples/dubbo/go-client/assembly/bin/load.sh
+++ b/examples/dubbo/go-client/assembly/bin/load.sh
@@ -23,8 +23,9 @@ else
     APP_NAME="APPLICATION_NAME.exe"
 fi
 
-export APP_CONF_FILE=${PROJECT_HOME}"TARGET_CONF_FILE"
+export CONF_CONSUMER_FILE_PATH=${PROJECT_HOME}"TARGET_CONF_FILE"
 export APP_LOG_CONF_FILE=${PROJECT_HOME}"TARGET_LOG_CONF_FILE"
+export CONF_DUBBO_CLIENT_FILE_PATH=${PROJECT_HOME}"TARGET_DUBBO_CONF_FILE"
 # export GOTRACEBACK=system
 # export GODEBUG=gctrace=1
 
diff --git a/examples/dubbo/go-client/assembly/common/app.properties b/examples/dubbo/go-client/assembly/common/app.properties
index a4fe0dc49c83e7c180408b02010ebf4bbefc98a9..07a1814d11b29ccbe5bfcab1e9375a8461e6f943 100644
--- a/examples/dubbo/go-client/assembly/common/app.properties
+++ b/examples/dubbo/go-client/assembly/common/app.properties
@@ -15,3 +15,4 @@ export BUILD_PACKAGE="app"
 
 export TARGET_CONF_FILE="conf/client.yml"
 export TARGET_LOG_CONF_FILE="conf/log.xml"
+export TARGET_DUBBO_CONF_FILE="conf/dubbo-client.yml"
diff --git a/examples/dubbo/go-client/assembly/common/build.sh b/examples/dubbo/go-client/assembly/common/build.sh
index 00763725b821e54dd0f378b0ec60757837067ef9..c52eb039beb6c8d05e11d3c3f1c16dc9a4baffab 100644
--- a/examples/dubbo/go-client/assembly/common/build.sh
+++ b/examples/dubbo/go-client/assembly/common/build.sh
@@ -69,6 +69,13 @@ else
     sed -i "s~TARGET_LOG_CONF_FILE~${TARGET_LOG_CONF_FILE}~g" ${BIN_DIR}/bin/*
 fi
 
+# modify TARGET_DUBBO_CONF_FILE
+if [ ${platform} == "Darwin" ]; then
+    sed -i "" "s~TARGET_DUBBO_CONF_FILE~${TARGET_DUBBO_CONF_FILE}~g" ${BIN_DIR}/bin/*
+else
+    sed -i "s~TARGET_DUBBO_CONF_FILE~${TARGET_DUBBO_CONF_FILE}~g" ${BIN_DIR}/bin/*
+fi
+
 cp -r profiles/${PROFILE}/* ${CONF_DIR}
 
 cd ${TARGET_FOLDER}
diff --git a/examples/dubbo/go-client/profiles/dev/client.yml b/examples/dubbo/go-client/profiles/dev/client.yml
index 9e7f65f36870b13ef9cd433e98480e0bf4ff8d4c..34196bd4233b11b6bdbdb405f94063dd590bf4a8 100644
--- a/examples/dubbo/go-client/profiles/dev/client.yml
+++ b/examples/dubbo/go-client/profiles/dev/client.yml
@@ -7,13 +7,8 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
@@ -24,12 +19,29 @@ application_config:
     owner : "ZX"
     environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        service : "com.ikurento.user.UserProvider"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "dubbo"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/dubbo/go-client/profiles/dev/dubbo-client.yml b/examples/dubbo/go-client/profiles/dev/dubbo-client.yml
new file mode 100644
index 0000000000000000000000000000000000000000..443031d567ec27fe4a2279f68f344618720c2098
--- /dev/null
+++ b/examples/dubbo/go-client/profiles/dev/dubbo-client.yml
@@ -0,0 +1,21 @@
+
+connection_number: 2
+heartbeat_period: "5s"
+session_timeout: "20s"
+fail_fast_timeout: "5s"
+pool_size: 64
+pool_ttl: 600
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "client"
diff --git a/examples/dubbo/go-client/profiles/release/client.yml b/examples/dubbo/go-client/profiles/release/client.yml
index 4ce8a0d68ebf17fd464b785f8f652aff52a04c09..34196bd4233b11b6bdbdb405f94063dd590bf4a8 100644
--- a/examples/dubbo/go-client/profiles/release/client.yml
+++ b/examples/dubbo/go-client/profiles/release/client.yml
@@ -7,13 +7,8 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
@@ -22,14 +17,31 @@ application_config:
     module : "dubbogo user-info client"
     version : "0.0.1"
     owner : "ZX"
-    environment : "product"
+    environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        service : "com.ikurento.user.UserProvider"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "dubbo"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/dubbo/go-client/profiles/release/dubbo-client.yml b/examples/dubbo/go-client/profiles/release/dubbo-client.yml
new file mode 100644
index 0000000000000000000000000000000000000000..443031d567ec27fe4a2279f68f344618720c2098
--- /dev/null
+++ b/examples/dubbo/go-client/profiles/release/dubbo-client.yml
@@ -0,0 +1,21 @@
+
+connection_number: 2
+heartbeat_period: "5s"
+session_timeout: "20s"
+fail_fast_timeout: "5s"
+pool_size: 64
+pool_ttl: 600
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "client"
diff --git a/examples/dubbo/go-client/profiles/test/client.yml b/examples/dubbo/go-client/profiles/test/client.yml
index 4e7e7a572bd2faea22fdee48f53dd50671eeb16b..34196bd4233b11b6bdbdb405f94063dd590bf4a8 100644
--- a/examples/dubbo/go-client/profiles/test/client.yml
+++ b/examples/dubbo/go-client/profiles/test/client.yml
@@ -7,13 +7,8 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
@@ -22,14 +17,31 @@ application_config:
     module : "dubbogo user-info client"
     version : "0.0.1"
     owner : "ZX"
-    environment : "test"
+    environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        service : "com.ikurento.user.UserProvider"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "dubbo"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/dubbo/go-client/profiles/test/dubbo-client.yml b/examples/dubbo/go-client/profiles/test/dubbo-client.yml
new file mode 100644
index 0000000000000000000000000000000000000000..443031d567ec27fe4a2279f68f344618720c2098
--- /dev/null
+++ b/examples/dubbo/go-client/profiles/test/dubbo-client.yml
@@ -0,0 +1,21 @@
+
+connection_number: 2
+heartbeat_period: "5s"
+session_timeout: "20s"
+fail_fast_timeout: "5s"
+pool_size: 64
+pool_ttl: 600
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "client"
diff --git a/examples/dubbo/go-server/app/config.go b/examples/dubbo/go-server/app/config.go
deleted file mode 100644
index 71d4aa5467402c6c0f8b78489b2c911334c0fa75..0000000000000000000000000000000000000000
--- a/examples/dubbo/go-server/app/config.go
+++ /dev/null
@@ -1,129 +0,0 @@
-package main
-
-import (
-	"fmt"
-	"github.com/dubbo/dubbo-go/plugins"
-	"io/ioutil"
-	"os"
-	"path"
-	"time"
-)
-
-import (
-	"github.com/AlexStocks/goext/log"
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-	yaml "gopkg.in/yaml.v2"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
-	"github.com/dubbo/dubbo-go/server"
-)
-
-const (
-	APP_CONF_FILE     string = "APP_CONF_FILE"
-	APP_LOG_CONF_FILE string = "APP_LOG_CONF_FILE"
-)
-
-var (
-	conf *ServerConfig
-)
-
-type (
-	ServerConfig struct {
-		// pprof
-		Pprof_Enabled bool `default:"false" yaml:"pprof_enabled"  json:"pprof_enabled,omitempty"`
-		Pprof_Port    int  `default:"10086"  yaml:"pprof_port" json:"pprof_port,omitempty"`
-
-		// transport & registry
-		Transport  string `default:"http"  yaml:"transport" json:"transport,omitempty"`
-		NetTimeout string `default:"100ms"  yaml:"net_timeout" json:"net_timeout,omitempty"` // in ms
-		netTimeout time.Duration
-		// application
-		Application_Config registry.ApplicationConfig `yaml:"application_config" json:"application_config,omitempty"`
-		// Registry_Address  string `default:"192.168.35.3:2181"`
-		Registry         string                     `default:"zookeeper"  yaml:"registry" json:"registry,omitempty"`
-		ZkRegistryConfig zookeeper.ZkRegistryConfig `yaml:"zk_registry_config" json:"zk_registry_config,omitempty"`
-
-		ServiceConfigType    string                   `default:"default" yaml:"service_config_type" json:"service_config_type,omitempty"`
-		ServiceConfigList    []registry.ServiceConfig `yaml:"-"`
-		ServiceConfigMapList []map[string]string      `yaml:"service_list" json:"service_list,omitempty"`
-		Server_List          []server.ServerConfig    `yaml:"server_list" json:"server_list,omitempty"`
-	}
-)
-
-func initServerConf() *ServerConfig {
-	var (
-		err      error
-		confFile string
-	)
-
-	confFile = os.Getenv(APP_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("application configure file name is nil"))
-		return nil
-	}
-	if path.Ext(confFile) != ".yml" {
-		panic(fmt.Sprintf("application configure file name{%v} suffix must be .yml", confFile))
-		return nil
-	}
-
-	conf = &ServerConfig{}
-	confFileStream, err := ioutil.ReadFile(confFile)
-	if err != nil {
-		panic(fmt.Sprintf("ioutil.ReadFile(file:%s) = error:%s", confFile, jerrors.ErrorStack(err)))
-		return nil
-	}
-	err = yaml.Unmarshal(confFileStream, conf)
-	if err != nil {
-		panic(fmt.Sprintf("yaml.Unmarshal() = error:%s", jerrors.ErrorStack(err)))
-		return nil
-	}
-	if conf.netTimeout, err = time.ParseDuration(conf.NetTimeout); err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(NetTimeout:%#v) = error:%s", conf.NetTimeout, err))
-		return nil
-	}
-	if conf.ZkRegistryConfig.Timeout, err = time.ParseDuration(conf.ZkRegistryConfig.TimeoutStr); err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Registry_Config.Timeout:%#v) = error:%s",
-			conf.ZkRegistryConfig.TimeoutStr, err))
-		return nil
-	}
-
-	// set designated service_config_type to default
-	plugins.SetDefaultProviderServiceConfig(conf.ServiceConfigType)
-	for _, service := range conf.ServiceConfigMapList {
-
-		svc := plugins.DefaultProviderServiceConfig()()
-		svc.SetProtocol(service["protocol"])
-		svc.SetService(service["service"])
-		conf.ServiceConfigList = append(conf.ServiceConfigList, svc)
-	}
-
-	gxlog.CInfo("config{%#v}\n", conf)
-
-	return conf
-}
-
-func configInit() error {
-	var (
-		confFile string
-	)
-
-	initServerConf()
-
-	confFile = os.Getenv(APP_LOG_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("log configure file name is nil"))
-		return nil
-	}
-	if path.Ext(confFile) != ".xml" {
-		panic(fmt.Sprintf("log configure file name{%v} suffix must be .xml", confFile))
-		return nil
-	}
-
-	log.LoadConfiguration(confFile)
-
-	return nil
-}
diff --git a/examples/dubbo/go-server/app/server.go b/examples/dubbo/go-server/app/server.go
index bfc039bf407fba0ab33c3c0f490fa903e01229ee..127212480a5f763f8531bd6907d1ce1f856fe74c 100644
--- a/examples/dubbo/go-server/app/server.go
+++ b/examples/dubbo/go-server/app/server.go
@@ -2,6 +2,7 @@ package main
 
 import (
 	"fmt"
+	"github.com/dubbogo/hessian2"
 	"net/http"
 	_ "net/http/pprof"
 	"os"
@@ -14,119 +15,49 @@ import (
 	"github.com/AlexStocks/goext/net"
 	"github.com/AlexStocks/goext/time"
 	log "github.com/AlexStocks/log4go"
-	"github.com/dubbogo/hessian2"
-	jerrors "github.com/juju/errors"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/dubbo"
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/protocol/dubbo"
+	_ "github.com/dubbo/go-for-apache-dubbo/protocol/jsonrpc"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/protocol"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/filter/imp"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/loadbalance"
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/support"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/zookeeper"
 )
 
 var (
 	survivalTimeout = int(3e9)
-	servo           *dubbo.Server
 )
 
+// they are necessary:
+// 		export CONF_PROVIDER_FILE_PATH="xxx"
+// 		export APP_LOG_CONF_FILE="xxx"
 func main() {
-	var (
-		err error
-	)
-
-	err = configInit()
-	if err != nil {
-		log.Error("configInit() = error{%#v}", err)
-		return
-	}
-	initProfiling()
 
+	// ------for hessian2------
 	hessian.RegisterJavaEnum(Gender(MAN))
 	hessian.RegisterJavaEnum(Gender(WOMAN))
-	hessian.RegisterPOJO(&DubboUser{})
-
-	servo = initServer()
-	err = servo.Register(&UserProvider{})
-	if err != nil {
-		panic(err)
-		return
-	}
-	servo.Start()
-
-	initSignal()
-}
-
-func initServer() *dubbo.Server {
-	var (
-		srv *dubbo.Server
-	)
-
-	if conf == nil {
-		panic(fmt.Sprintf("conf is nil"))
-		return nil
-	}
-
-	// registry
-
-	regs, err := plugins.PluggableRegistries[conf.Registry](
-		registry.WithDubboType(registry.PROVIDER),
-		registry.WithApplicationConf(conf.Application_Config),
-		zookeeper.WithRegistryConf(conf.ZkRegistryConfig),
-	)
+	hessian.RegisterPOJO(&User{})
+	// ------------
 
-	if err != nil || regs == nil {
-		panic(fmt.Sprintf("fail to init registry.Registy, err:%s", jerrors.ErrorStack(err)))
-		return nil
+	_, proMap := support.Load()
+	if proMap == nil {
+		panic("proMap is nil")
 	}
 
-	// generate server config
-	serverConfig := make([]dubbo.ServerConfig, len(conf.Server_List))
-	for i := 0; i < len(conf.Server_List); i++ {
-		serverConfig[i] = dubbo.ServerConfig{
-			SessionNumber:   700,
-			FailFastTimeout: "5s",
-			SessionTimeout:  "20s",
-			GettySessionParam: dubbo.GettySessionParam{
-				CompressEncoding: false, // 必须false
-				TcpNoDelay:       true,
-				KeepAlivePeriod:  "120s",
-				TcpRBufSize:      262144,
-				TcpKeepAlive:     true,
-				TcpWBufSize:      65536,
-				PkgRQSize:        1024,
-				PkgWQSize:        512,
-				TcpReadTimeout:   "1s",
-				TcpWriteTimeout:  "5s",
-				WaitTimeout:      "1s",
-				MaxMsgLen:        1024,
-				SessionName:      "server",
-			},
-		}
-		serverConfig[i].IP = conf.Server_List[i].IP
-		serverConfig[i].Port = conf.Server_List[i].Port
-		serverConfig[i].Protocol = conf.Server_List[i].Protocol
-	}
-
-	// provider
-	srv = dubbo.NewServer(
-		dubbo.Registry(regs),
-		dubbo.ConfList(serverConfig),
-		dubbo.ServiceConfList(conf.ServiceConfigList),
-	)
-
-	return srv
-}
+	initProfiling()
 
-func uninitServer() {
-	if servo != nil {
-		servo.Stop()
-	}
-	log.Close()
+	initSignal()
 }
 
 func initProfiling() {
-	if !conf.Pprof_Enabled {
+	if !support.GetProviderConfig().Pprof_Enabled {
 		return
 	}
 	const (
@@ -142,7 +73,7 @@ func initProfiling() {
 	if err != nil {
 		panic("cat not get local ip!")
 	}
-	addr = ip + ":" + strconv.Itoa(conf.Pprof_Port)
+	addr = ip + ":" + strconv.Itoa(support.GetProviderConfig().Pprof_Port)
 	log.Info("App Profiling startup on address{%v}", addr+PprofPath)
 
 	go func() {
@@ -159,7 +90,7 @@ func initSignal() {
 		log.Info("get signal %s", sig.String())
 		switch sig {
 		case syscall.SIGHUP:
-		// reload()
+			// reload()
 		default:
 			go gxtime.Future(survivalTimeout, func() {
 				log.Warn("app exit now by force...")
@@ -167,7 +98,6 @@ func initSignal() {
 			})
 
 			// 要么fastFailTimeout时间内执行完毕下面的逻辑然后程序退出,要么执行上面的超时函数程序强行退出
-			uninitServer()
 			fmt.Println("provider app exit now...")
 			return
 		}
diff --git a/examples/dubbo/go-server/app/user.go b/examples/dubbo/go-server/app/user.go
index fa4f9f992eb8d50ee962527c6b7551125ec2da02..a6a51bb9b92a2cf69359fab69b5a444e3bb06a34 100644
--- a/examples/dubbo/go-server/app/user.go
+++ b/examples/dubbo/go-server/app/user.go
@@ -1,7 +1,6 @@
 package main
 
 import (
-	// "encoding/json"
 	"context"
 	"fmt"
 	"strconv"
@@ -13,8 +12,16 @@ import (
 	"github.com/dubbogo/hessian2"
 )
 
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+)
+
 type Gender hessian.JavaEnum
 
+func init() {
+	support.SetProService(new(UserProvider))
+}
+
 const (
 	MAN hessian.JavaEnum = iota
 	WOMAN
@@ -53,7 +60,7 @@ func (g Gender) EnumValue(s string) hessian.JavaEnum {
 }
 
 type (
-	DubboUser struct {
+	User struct {
 		// !!! Cannot define lowercase names of variable
 		Id   string
 		Name string
@@ -63,42 +70,42 @@ type (
 	}
 
 	UserProvider struct {
-		user map[string]DubboUser
+		user map[string]User
 	}
 )
 
-func (u DubboUser) String() string {
-	return fmt.Sprintf(
-		"User{Id:%s, Name:%s, Age:%d, Time:%s, Sex:%s}",
-		u.Id, u.Name, u.Age, u.Time, u.Sex,
-	)
-}
-
-func (DubboUser) JavaClassName() string {
-	return "com.ikurento.user.User"
-}
-
 var (
-	DefaultUser = DubboUser{
+	DefaultUser = User{
 		Id: "0", Name: "Alex Stocks", Age: 31,
 		Sex: Gender(MAN),
 	}
 
-	userMap = UserProvider{user: make(map[string]DubboUser)}
+	userMap = UserProvider{user: make(map[string]User)}
 )
 
 func init() {
-	//DefaultUser.Sex = DefaultUser.sex.String()
 	userMap.user["A000"] = DefaultUser
-	userMap.user["A001"] = DubboUser{Id: "001", Name: "ZhangSheng", Age: 18, Sex: Gender(MAN)}
-	userMap.user["A002"] = DubboUser{Id: "002", Name: "Lily", Age: 20, Sex: Gender(WOMAN)}
-	userMap.user["A003"] = DubboUser{Id: "113", Name: "Moorse", Age: 30, Sex: Gender(WOMAN)}
+	userMap.user["A001"] = User{Id: "001", Name: "ZhangSheng", Age: 18, Sex: Gender(MAN)}
+	userMap.user["A002"] = User{Id: "002", Name: "Lily", Age: 20, Sex: Gender(WOMAN)}
+	userMap.user["A003"] = User{Id: "113", Name: "Moorse", Age: 30, Sex: Gender(WOMAN)}
 	for k, v := range userMap.user {
+		v.Time = time.Now()
 		userMap.user[k] = v
 	}
 }
 
-func (u *UserProvider) getUser(userId string) (*DubboUser, error) {
+func (u User) String() string {
+	return fmt.Sprintf(
+		"User{Id:%s, Name:%s, Age:%d, Time:%s, Sex:%s}",
+		u.Id, u.Name, u.Age, u.Time, u.Sex,
+	)
+}
+
+func (u User) JavaClassName() string {
+	return "com.ikurento.user.User"
+}
+
+func (u *UserProvider) getUser(userId string) (*User, error) {
 	if user, ok := userMap.user[userId]; ok {
 		return &user, nil
 	}
@@ -106,13 +113,10 @@ func (u *UserProvider) getUser(userId string) (*DubboUser, error) {
 	return nil, fmt.Errorf("invalid user id:%s", userId)
 }
 
-/*
-	!!! req must be []interface{}
-*/
-func (u *UserProvider) GetUser(ctx context.Context, req []interface{}, rsp *DubboUser) error {
+func (u *UserProvider) GetUser(ctx context.Context, req []interface{}, rsp *User) error {
 	var (
 		err  error
-		user *DubboUser
+		user *User
 	)
 
 	gxlog.CInfo("req:%#v", req)
@@ -120,6 +124,11 @@ func (u *UserProvider) GetUser(ctx context.Context, req []interface{}, rsp *Dubb
 	if err == nil {
 		*rsp = *user
 		gxlog.CInfo("rsp:%#v", rsp)
+		// s, _ := json.Marshal(rsp)
+		// fmt.Println("hello0:", string(s))
+
+		// s, _ = json.Marshal(*rsp)
+		// fmt.Println("hello1:", string(s))
 	}
 	return err
 }
diff --git a/examples/dubbo/go-server/assembly/bin/load.sh b/examples/dubbo/go-server/assembly/bin/load.sh
index e202ff65f436f08191ae5364378f659de858777a..f89c45255f4184c1ceece951c58252e56c4bd984 100644
--- a/examples/dubbo/go-server/assembly/bin/load.sh
+++ b/examples/dubbo/go-server/assembly/bin/load.sh
@@ -20,8 +20,9 @@ if [[ ${OS_NAME} != "Windows" ]]; then
     PROJECT_HOME=${PROJECT_HOME}"/"
 fi
 
-export APP_CONF_FILE=${PROJECT_HOME}"TARGET_CONF_FILE"
+export CONF_PROVIDER_FILE_PATH=${PROJECT_HOME}"TARGET_CONF_FILE"
 export APP_LOG_CONF_FILE=${PROJECT_HOME}"TARGET_LOG_CONF_FILE"
+export CONF_DUBBO_SERVER_FILE_PATH=${PROJECT_HOME}"TARGET_DUBBO_CONF_FILE"
 
 usage() {
     echo "Usage: $0 start"
diff --git a/examples/dubbo/go-server/assembly/common/app.properties b/examples/dubbo/go-server/assembly/common/app.properties
index d230d5efc4ee84c4a99e1b27e7b49d97046d91a3..53eb3a553864e0aea2f103a579ed6d1312b433d0 100644
--- a/examples/dubbo/go-server/assembly/common/app.properties
+++ b/examples/dubbo/go-server/assembly/common/app.properties
@@ -15,3 +15,4 @@ BUILD_PACKAGE="app"
 
 TARGET_CONF_FILE="conf/server.yml"
 TARGET_LOG_CONF_FILE="conf/log.xml"
+TARGET_DUBBO_CONF_FILE="conf/dubbo-server.yml"
diff --git a/examples/dubbo/go-server/assembly/common/build.sh b/examples/dubbo/go-server/assembly/common/build.sh
index a41fbbac321b74849d71b28a65f8b7c5de13cf0f..ddc40e4e0c36432eb0a2a73f06f7dd6b4343f354 100644
--- a/examples/dubbo/go-server/assembly/common/build.sh
+++ b/examples/dubbo/go-server/assembly/common/build.sh
@@ -65,6 +65,12 @@ if [ "$(uname)" == "Darwin" ]; then
 else
     sed -i "s~TARGET_LOG_CONF_FILE~${TARGET_LOG_CONF_FILE}~g" ${BIN_DIR}/bin/*
 fi
+#modify TARGET_DUBBO_CONF_FILE
+if [ "$(uname)" == "Darwin" ]; then
+    sed -i "" "s~TARGET_DUBBO_CONF_FILE~${TARGET_DUBBO_CONF_FILE}~g" ${BIN_DIR}/bin/*
+else
+    sed -i "s~TARGET_DUBBO_CONF_FILE~${TARGET_DUBBO_CONF_FILE}~g" ${BIN_DIR}/bin/*
+fi
 
 cp -r profiles/${PROFILE}/* ${CONF_DIR}
 
diff --git a/examples/dubbo/go-server/profiles/dev/dubbo-server.yml b/examples/dubbo/go-server/profiles/dev/dubbo-server.yml
new file mode 100644
index 0000000000000000000000000000000000000000..ae19827a7d0a58dd1573ae5644630ca5f644dfa8
--- /dev/null
+++ b/examples/dubbo/go-server/profiles/dev/dubbo-server.yml
@@ -0,0 +1,18 @@
+
+session_number: 700
+fail_fast_timeout: "5s"
+session_timeout: "20s"
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "server"
diff --git a/examples/dubbo/go-server/profiles/dev/server.yml b/examples/dubbo/go-server/profiles/dev/server.yml
index 3eb4df6112275fd9ffebe7a416c39a1259867512..f055b1610caf5333b6fa2bd50c91a18088556386 100644
--- a/examples/dubbo/go-server/profiles/dev/server.yml
+++ b/examples/dubbo/go-server/profiles/dev/server.yml
@@ -8,6 +8,7 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
     organization : "ikurento.com"
@@ -17,23 +18,42 @@ application_config:
     owner : "ZX"
     environment : "dev"
 
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        # 如果是127.0.0.1, java-client将无法连接到go-server
-        ip : "192.168.56.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "dubbo"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "dubbo"
+    # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+-   name: "dubbo"
+    ip : "192.168.56.1"
+    port : 20000
+#-   name: "jsonrpc"
+#    ip: "127.0.0.1"
+#    port: 20001
+
diff --git a/examples/dubbo/go-server/profiles/release/dubbo-server.yml b/examples/dubbo/go-server/profiles/release/dubbo-server.yml
new file mode 100644
index 0000000000000000000000000000000000000000..ae19827a7d0a58dd1573ae5644630ca5f644dfa8
--- /dev/null
+++ b/examples/dubbo/go-server/profiles/release/dubbo-server.yml
@@ -0,0 +1,18 @@
+
+session_number: 700
+fail_fast_timeout: "5s"
+session_timeout: "20s"
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "server"
diff --git a/examples/dubbo/go-server/profiles/release/server.yml b/examples/dubbo/go-server/profiles/release/server.yml
index e61274ea0977f2a193572cddc2ec996e07c220b0..f055b1610caf5333b6fa2bd50c91a18088556386 100644
--- a/examples/dubbo/go-server/profiles/release/server.yml
+++ b/examples/dubbo/go-server/profiles/release/server.yml
@@ -8,6 +8,7 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
     organization : "ikurento.com"
@@ -15,24 +16,44 @@ application_config:
     module : "dubbogo user-info server"
     version : "0.0.1"
     owner : "ZX"
-    environment : "product"
-
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        ip : "127.0.0.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "dubbo"
+    environment : "dev"
+
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "dubbo"
+    # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+-   name: "dubbo"
+    ip : "192.168.56.1"
+    port : 20000
+#-   name: "jsonrpc"
+#    ip: "127.0.0.1"
+#    port: 20001
+
diff --git a/examples/dubbo/go-server/profiles/test/dubbo-server.yml b/examples/dubbo/go-server/profiles/test/dubbo-server.yml
new file mode 100644
index 0000000000000000000000000000000000000000..ae19827a7d0a58dd1573ae5644630ca5f644dfa8
--- /dev/null
+++ b/examples/dubbo/go-server/profiles/test/dubbo-server.yml
@@ -0,0 +1,18 @@
+
+session_number: 700
+fail_fast_timeout: "5s"
+session_timeout: "20s"
+getty_session_param:
+  compress_encoding: false
+  tcp_no_delay: true
+  tcp_keep_alive: true
+  keep_alive_period: "120s"
+  tcp_r_buf_size: 262144
+  tcp_w_buf_size: 65536
+  pkg_rq_size: 1024
+  pkg_wq_size: 512
+  tcp_read_timeout: "1s"
+  tcp_write_timeout: "5s"
+  wait_timeout: "1s"
+  max_msg_len: 1024
+  session_name: "server"
diff --git a/examples/dubbo/go-server/profiles/test/server.yml b/examples/dubbo/go-server/profiles/test/server.yml
index cc73b9b642f2ba7f36c781f72e153b0547e7e8ce..f055b1610caf5333b6fa2bd50c91a18088556386 100644
--- a/examples/dubbo/go-server/profiles/test/server.yml
+++ b/examples/dubbo/go-server/profiles/test/server.yml
@@ -8,6 +8,7 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
     organization : "ikurento.com"
@@ -15,24 +16,44 @@ application_config:
     module : "dubbogo user-info server"
     version : "0.0.1"
     owner : "ZX"
-    environment : "test"
-
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "dubbo"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        ip : "127.0.0.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "dubbo"
+    environment : "dev"
+
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "dubbo"
+    # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+-   name: "dubbo"
+    ip : "192.168.56.1"
+    port : 20000
+#-   name: "jsonrpc"
+#    ip: "127.0.0.1"
+#    port: 20001
+
diff --git a/examples/jsonrpc/go-client/app/client.go b/examples/jsonrpc/go-client/app/client.go
index acbe035a8e6fe95c4c22f09f224138854f4e2169..b2771c7e4ca95081b7bf9a2f04c0f41868493ddf 100644
--- a/examples/jsonrpc/go-client/app/client.go
+++ b/examples/jsonrpc/go-client/app/client.go
@@ -1,6 +1,7 @@
 package main
 
 import (
+	"context"
 	"fmt"
 	"net/http"
 	_ "net/http/pprof"
@@ -15,117 +16,58 @@ import (
 	"github.com/AlexStocks/goext/log"
 	"github.com/AlexStocks/goext/net"
 	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/client/invoker"
-	"github.com/dubbo/dubbo-go/examples"
-	"github.com/dubbo/dubbo-go/jsonrpc"
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/public"
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
+	_ "github.com/dubbo/go-for-apache-dubbo/protocol/jsonrpc"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/protocol"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/filter/imp"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/loadbalance"
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/support"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/zookeeper"
+
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
 )
 
 var (
 	survivalTimeout int = 10e9
-	clientInvoker   *invoker.Invoker
 )
 
+// they are necessary:
+// 		export CONF_CONSUMER_FILE_PATH="xxx"
+// 		export APP_LOG_CONF_FILE="xxx"
 func main() {
 
-	clientConfig := examples.InitClientConfig()
-
-	initProfiling(clientConfig)
-	initClient(clientConfig)
+	conMap, _ := support.Load()
+	if conMap == nil {
+		panic("conMap is nil")
+	}
 
-	time.Sleep(10e9)
+	initProfiling()
 
-	gxlog.CInfo("\n\n\nstart to test jsonrpc")
-	testJsonrpc(clientConfig, "A003", "GetUser")
 	time.Sleep(3e9)
 
-	gxlog.CInfo("\n\n\nstart to test jsonrpc illegal method")
-
-	testJsonrpc(clientConfig, "A003", "GetUser1")
-
-	initSignal()
-}
-
-func initClient(clientConfig *examples.ClientConfig) {
-	var (
-		codecType public.CodecType
-	)
-
-	if clientConfig == nil {
-		panic(fmt.Sprintf("clientConfig is nil"))
-		return
-	}
-
-	// registry
-	clientRegistry, err := plugins.PluggableRegistries[clientConfig.Registry](
-		registry.WithDubboType(registry.CONSUMER),
-		registry.WithApplicationConf(clientConfig.Application_Config),
-		zookeeper.WithRegistryConf(clientConfig.ZkRegistryConfig),
-	)
+	gxlog.CInfo("\n\n\nstart to test jsonrpc")
+	user := &JsonRPCUser{}
+	err := conMap["com.ikurento.user.UserProvider"].GetRPCService().(*UserProvider).GetUser(context.TODO(), []interface{}{"A003"}, user)
 	if err != nil {
-		panic(fmt.Sprintf("fail to init registry.Registy, err:%s", jerrors.ErrorStack(err)))
-		return
+		panic(err)
 	}
+	gxlog.CInfo("response result: %v", user)
 
-	// consumer
-	clientConfig.RequestTimeout, err = time.ParseDuration(clientConfig.Request_Timeout)
-	if err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Request_Timeout{%#v}) = error{%v}",
-			clientConfig.Request_Timeout, err))
-		return
-	}
-	clientConfig.ConnectTimeout, err = time.ParseDuration(clientConfig.Connect_Timeout)
+	gxlog.CInfo("\n\n\nstart to test jsonrpc illegal method")
+	err = conMap["com.ikurento.user.UserProvider"].GetRPCService().(*UserProvider).GetUser1(context.TODO(), []interface{}{"A003"}, user)
 	if err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Connect_Timeout{%#v}) = error{%v}",
-			clientConfig.Connect_Timeout, err))
-		return
-	}
-
-	for idx := range clientConfig.ServiceConfigList {
-		codecType = public.GetCodecType(clientConfig.ServiceConfigList[idx].Protocol())
-		if codecType == public.CODECTYPE_UNKNOWN {
-			panic(fmt.Sprintf("unknown protocol %s", clientConfig.ServiceConfigList[idx].Protocol()))
-		}
-	}
-
-	for _, service := range clientConfig.ServiceConfigList {
-		err = clientRegistry.Register(service)
-		if err != nil {
-			panic(fmt.Sprintf("registry.Register(service{%#v}) = error{%v}", service, jerrors.ErrorStack(err)))
-			return
-		}
+		panic(err)
 	}
 
-	//read the client lb config in config.yml
-	configClientLB := plugins.PluggableLoadbalance[clientConfig.ClientLoadBalance]()
-
-	//init http client & init invoker
-	clt := jsonrpc.NewHTTPClient(
-		&jsonrpc.HTTPOptions{
-			HandshakeTimeout: clientConfig.ConnectTimeout,
-			HTTPTimeout:      clientConfig.RequestTimeout,
-		},
-	)
-
-	clientInvoker, err = invoker.NewInvoker(clientRegistry,
-		invoker.WithHttpClient(clt),
-		invoker.WithLBSelector(configClientLB))
-
-}
-
-func uninitClient() {
-	log.Close()
+	initSignal()
 }
 
-func initProfiling(clientConfig *examples.ClientConfig) {
-	if !clientConfig.Pprof_Enabled {
+func initProfiling() {
+	if !support.GetProviderConfig().Pprof_Enabled {
 		return
 	}
 	const (
@@ -141,7 +83,7 @@ func initProfiling(clientConfig *examples.ClientConfig) {
 	if err != nil {
 		panic("cat not get local ip!")
 	}
-	addr = ip + ":" + strconv.Itoa(clientConfig.Pprof_Port)
+	addr = ip + ":" + strconv.Itoa(support.GetProviderConfig().Pprof_Port)
 	log.Info("App Profiling startup on address{%v}", addr+PprofPath)
 
 	go func() {
@@ -167,7 +109,6 @@ func initSignal() {
 			})
 
 			// 要么fastFailTimeout时间内执行完毕下面的逻辑然后程序退出,要么执行上面的超时函数程序强行退出
-			uninitClient()
 			fmt.Println("app exit now...")
 			return
 		}
diff --git a/examples/jsonrpc/go-client/app/test.go b/examples/jsonrpc/go-client/app/test.go
deleted file mode 100644
index 967d69696bd953bdc442520fe1efcb2d10827c9f..0000000000000000000000000000000000000000
--- a/examples/jsonrpc/go-client/app/test.go
+++ /dev/null
@@ -1,66 +0,0 @@
-package main
-
-import (
-	"context"
-	"fmt"
-	_ "net/http/pprof"
-)
-
-import (
-	"github.com/AlexStocks/goext/log"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/examples"
-	"github.com/dubbo/dubbo-go/public"
-)
-
-func testJsonrpc(clientConfig *examples.ClientConfig, userKey string, method string) {
-	var (
-		err        error
-		svc        string
-		serviceIdx int
-		user       *JsonRPCUser
-		ctx        context.Context
-		req        client.Request
-	)
-
-	serviceIdx = -1
-	svc = "com.ikurento.user.UserProvider"
-	for i := range clientConfig.ServiceConfigList {
-		if clientConfig.ServiceConfigList[i].Service() == svc && clientConfig.ServiceConfigList[i].Protocol() == public.CODECTYPE_JSONRPC.String() {
-			serviceIdx = i
-			break
-		}
-	}
-	if serviceIdx == -1 {
-		panic(fmt.Sprintf("can not find service in config service list:%#v", clientConfig.ServiceConfigList))
-	}
-
-	// Create request
-	// gxlog.CInfo("jsonrpc selected service %#v", clientConfig.ServiceConfigList[serviceIdx])
-
-	// Attention the last parameter : []UserKey{userKey}
-	req, err = clientInvoker.HttpClient.NewRequest(clientConfig.ServiceConfigList[serviceIdx], method, []string{userKey})
-
-	if err != nil {
-		panic(err)
-	}
-
-	ctx = context.WithValue(context.Background(), public.DUBBOGO_CTX_KEY, map[string]string{
-		"X-Proxy-Id": "dubbogo",
-		"X-Services": svc,
-		"X-Method":   method,
-	})
-
-	user = new(JsonRPCUser)
-
-	err = clientInvoker.HttpCall(ctx, 1, req, user)
-	if err != nil {
-		panic(err)
-	} else {
-		gxlog.CInfo("response result:%s", user)
-	}
-
-}
diff --git a/examples/jsonrpc/go-client/app/user.go b/examples/jsonrpc/go-client/app/user.go
index 4ff080343887e9027d099639a9580ed4d93dfe3a..b5d189adcabbcb345157129654f80bfb138ffabd 100644
--- a/examples/jsonrpc/go-client/app/user.go
+++ b/examples/jsonrpc/go-client/app/user.go
@@ -1,6 +1,7 @@
 package main
 
 import (
+	"context"
 	"fmt"
 )
 
@@ -8,6 +9,14 @@ import (
 	"github.com/AlexStocks/goext/time"
 )
 
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+)
+
+func init() {
+	support.SetConService(new(UserProvider))
+}
+
 type JsonRPCUser struct {
 	ID   string `json:"id"`
 	Name string `json:"name"`
@@ -22,3 +31,16 @@ func (u JsonRPCUser) String() string {
 		u.ID, u.Name, u.Age, gxtime.YMDPrint(int(u.Time), 0), u.Sex,
 	)
 }
+
+type UserProvider struct {
+	GetUser  func(ctx context.Context, req []interface{}, rsp *JsonRPCUser) error
+	GetUser1 func(ctx context.Context, req []interface{}, rsp *JsonRPCUser) error
+}
+
+func (u *UserProvider) Service() string {
+	return "com.ikurento.user.UserProvider"
+}
+
+func (u *UserProvider) Version() string {
+	return ""
+}
diff --git a/examples/jsonrpc/go-client/assembly/bin/load.sh b/examples/jsonrpc/go-client/assembly/bin/load.sh
index 72127283082e62d9d2bdf2a4b4934db56674535b..63bb08a3b1ff8f0790df1b2a04285bbfefe10407 100644
--- a/examples/jsonrpc/go-client/assembly/bin/load.sh
+++ b/examples/jsonrpc/go-client/assembly/bin/load.sh
@@ -23,7 +23,7 @@ else
     APP_NAME="APPLICATION_NAME.exe"
 fi
 
-export APP_CONF_FILE=${PROJECT_HOME}"TARGET_CONF_FILE"
+export CONF_CONSUMER_FILE_PATH=${PROJECT_HOME}"TARGET_CONF_FILE"
 export APP_LOG_CONF_FILE=${PROJECT_HOME}"TARGET_LOG_CONF_FILE"
 # export GOTRACEBACK=system
 # export GODEBUG=gctrace=1
diff --git a/examples/jsonrpc/go-client/profiles/dev/client.yml b/examples/jsonrpc/go-client/profiles/dev/client.yml
index 47a364f90dc90ba03914d2e6bfffd690368edad1..839d9489127fa91d610ae2890492caf77177ad99 100644
--- a/examples/jsonrpc/go-client/profiles/dev/client.yml
+++ b/examples/jsonrpc/go-client/profiles/dev/client.yml
@@ -7,13 +7,8 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
@@ -24,13 +19,29 @@ application_config:
     owner : "ZX"
     environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
 
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        service : "com.ikurento.user.UserProvider"
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "jsonrpc"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/jsonrpc/go-client/profiles/release/client.yml b/examples/jsonrpc/go-client/profiles/release/client.yml
index fe4828022bf56344896375716328e5cd7f4c96ae..3b923bc48cdf40de7c6c7a91bcd1f9bfd85a6dfa 100644
--- a/examples/jsonrpc/go-client/profiles/release/client.yml
+++ b/examples/jsonrpc/go-client/profiles/release/client.yml
@@ -7,30 +7,41 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
-    organization : "ikurento.com"
-    name  : "BDTService"
-    module : "dubbogo user-info client"
-    version : "0.0.1"
-    owner : "ZX"
-    environment : "product"
+  organization : "ikurento.com"
+  name  : "BDTService"
+  module : "dubbogo user-info client"
+  version : "0.0.1"
+  owner : "ZX"
+  environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
 
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        service : "com.ikurento.user.UserProvider"
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "jsonrpc"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/jsonrpc/go-client/profiles/test/client.yml b/examples/jsonrpc/go-client/profiles/test/client.yml
index 71d48fd2a06d9580d476fb882b5d0fc22eef09e3..3b923bc48cdf40de7c6c7a91bcd1f9bfd85a6dfa 100644
--- a/examples/jsonrpc/go-client/profiles/test/client.yml
+++ b/examples/jsonrpc/go-client/profiles/test/client.yml
@@ -7,30 +7,41 @@ pprof_port : 10086
 # client
 request_timeout : "3500ms"
 net_io_timeout : "2s"
-retries : 1
 # connect timeout
 connect_timeout : "100ms"
-selector : "cache"
-selector_ttl : "10m"
-registry : "zookeeper"
-client_load_balance: "round_robin"
 
 # application config
 application_config:
-    organization : "ikurento.com"
-    name  : "BDTService"
-    module : "dubbogo user-info client"
-    version : "0.0.1"
-    owner : "ZX"
-    environment : "test"
+  organization : "ikurento.com"
+  name  : "BDTService"
+  module : "dubbogo user-info client"
+  version : "0.0.1"
+  owner : "ZX"
+  environment : "dev"
 
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
 
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        service : "com.ikurento.user.UserProvider"
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+references:
+- registries :
+  - "hangzhouzk"
+  - "shanghaizk"
+
+  protocol : "jsonrpc"
+  interface : "com.ikurento.user.UserProvider"
+  cluster: "failover"
+  methods :
+  - name: "GetUser"
+    retries: 3
diff --git a/examples/jsonrpc/go-server/app/config.go b/examples/jsonrpc/go-server/app/config.go
deleted file mode 100644
index 16064fca39871f8acd47061fe64124854d22dadf..0000000000000000000000000000000000000000
--- a/examples/jsonrpc/go-server/app/config.go
+++ /dev/null
@@ -1,132 +0,0 @@
-package main
-
-import (
-	"fmt"
-	"io/ioutil"
-	"os"
-	"path"
-	"time"
-)
-
-import (
-	"github.com/AlexStocks/goext/log"
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-	"gopkg.in/yaml.v2"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
-	"github.com/dubbo/dubbo-go/server"
-)
-
-const (
-	APP_CONF_FILE     string = "APP_CONF_FILE"
-	APP_LOG_CONF_FILE string = "APP_LOG_CONF_FILE"
-)
-
-var (
-	conf *ServerConfig
-)
-
-type (
-	ServerConfig struct {
-		// pprof
-		Pprof_Enabled bool `default:"false" yaml:"pprof_enabled"  json:"pprof_enabled,omitempty"`
-		Pprof_Port    int  `default:"10086"  yaml:"pprof_port" json:"pprof_port,omitempty"`
-
-		// transport & registry
-		Transport  string `default:"http"  yaml:"transport" json:"transport,omitempty"`
-		NetTimeout string `default:"100ms"  yaml:"net_timeout" json:"net_timeout,omitempty"` // in ms
-		netTimeout time.Duration
-		// application
-		Application_Config registry.ApplicationConfig `yaml:"application_config" json:"application_config,omitempty"`
-		// Registry_Address  string `default:"192.168.35.3:2181"`
-		Registry         string                     `default:"zookeeper"  yaml:"registry" json:"registry,omitempty"`
-		ZkRegistryConfig zookeeper.ZkRegistryConfig `yaml:"zk_registry_config" json:"zk_registry_config,omitempty"`
-
-		ServiceConfigType    string                           `default:"default" yaml:"service_config_type" json:"service_config_type,omitempty"`
-		ServiceConfigList    []registry.ProviderServiceConfig `yaml:"-"`
-		ServiceConfigMapList []map[string]string              `yaml:"service_list" json:"service_list,omitempty"`
-
-		ServerConfigList []server.ServerConfig `yaml:"server_list" json:"server_list,omitempty"`
-	}
-)
-
-func initServerConf() *ServerConfig {
-	var (
-		err      error
-		confFile string
-	)
-
-	confFile = os.Getenv(APP_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("application configure file name is nil"))
-		return nil
-	}
-	if path.Ext(confFile) != ".yml" {
-		panic(fmt.Sprintf("application configure file name{%v} suffix must be .yml", confFile))
-		return nil
-	}
-
-	conf = &ServerConfig{}
-	confFileStream, err := ioutil.ReadFile(confFile)
-	if err != nil {
-		panic(fmt.Sprintf("ioutil.ReadFile(file:%s) = error:%s", confFile, jerrors.ErrorStack(err)))
-		return nil
-	}
-	err = yaml.Unmarshal(confFileStream, conf)
-
-	//动态加载service config
-	//设置默认ProviderServiceConfig类
-	plugins.SetDefaultProviderServiceConfig(conf.ServiceConfigType)
-	for _, service := range conf.ServiceConfigMapList {
-
-		svc := plugins.DefaultProviderServiceConfig()()
-		svc.SetProtocol(service["protocol"])
-		svc.SetService(service["service"])
-		conf.ServiceConfigList = append(conf.ServiceConfigList, svc)
-	}
-	//动态加载service config  end
-	if err != nil {
-		panic(fmt.Sprintf("yaml.Unmarshal() = error:%s", jerrors.ErrorStack(err)))
-		return nil
-	}
-	if conf.netTimeout, err = time.ParseDuration(conf.NetTimeout); err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(NetTimeout:%#v) = error:%s", conf.NetTimeout, err))
-		return nil
-	}
-	if conf.ZkRegistryConfig.Timeout, err = time.ParseDuration(conf.ZkRegistryConfig.TimeoutStr); err != nil {
-		panic(fmt.Sprintf("time.ParseDuration(Registry_Config.Timeout:%#v) = error:%s",
-			conf.ZkRegistryConfig.TimeoutStr, err))
-		return nil
-	}
-
-	gxlog.CInfo("config{%#v}\n", conf)
-
-	return conf
-}
-
-func configInit() error {
-	var (
-		confFile string
-	)
-
-	initServerConf()
-
-	confFile = os.Getenv(APP_LOG_CONF_FILE)
-	if confFile == "" {
-		panic(fmt.Sprintf("log configure file name is nil"))
-		return nil
-	}
-	if path.Ext(confFile) != ".xml" {
-		panic(fmt.Sprintf("log configure file name{%v} suffix must be .xml", confFile))
-		return nil
-	}
-
-	log.LoadConfiguration(confFile)
-
-	return nil
-}
diff --git a/examples/jsonrpc/go-server/app/server.go b/examples/jsonrpc/go-server/app/server.go
index e355e268927051a4e7a0d786194ed5ded5267797..8e4ec3bc62ddfcde643f2b2b767c7b1b90817a6b 100644
--- a/examples/jsonrpc/go-server/app/server.go
+++ b/examples/jsonrpc/go-server/app/server.go
@@ -14,86 +14,42 @@ import (
 	"github.com/AlexStocks/goext/net"
 	"github.com/AlexStocks/goext/time"
 	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/jsonrpc"
-	"github.com/dubbo/dubbo-go/plugins"
-	registry2 "github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/registry/zookeeper"
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/protocol/jsonrpc"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/protocol"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/filter/imp"
+
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/loadbalance"
+	_ "github.com/dubbo/go-for-apache-dubbo/cluster/support"
+	_ "github.com/dubbo/go-for-apache-dubbo/registry/zookeeper"
 )
 
 var (
 	survivalTimeout = int(3e9)
-	servo           *jsonrpc.Server
 )
 
+// they are necessary:
+// 		export CONF_PROVIDER_FILE_PATH="xxx"
+// 		export APP_LOG_CONF_FILE="xxx"
 func main() {
-	var (
-		err error
-	)
 
-	err = configInit()
-	if err != nil {
-		log.Error("configInit() = error{%#v}", err)
-		return
+	_, proMap := support.Load()
+	if proMap == nil {
+		panic("proMap is nil")
 	}
-	initProfiling()
 
-	servo = initServer()
-	err = servo.Handle(&UserProvider{})
-	if err != nil {
-		panic(err)
-		return
-	}
-	servo.Start()
+	initProfiling()
 
 	initSignal()
 }
 
-func initServer() *jsonrpc.Server {
-	var (
-		srv *jsonrpc.Server
-	)
-
-	if conf == nil {
-		panic(fmt.Sprintf("conf is nil"))
-		return nil
-	}
-
-	// registry
-
-	registry, err := plugins.PluggableRegistries[conf.Registry](
-		registry2.WithDubboType(registry2.PROVIDER),
-		registry2.WithApplicationConf(conf.Application_Config),
-		zookeeper.WithRegistryConf(conf.ZkRegistryConfig),
-	)
-
-	if err != nil || registry == nil {
-		panic(fmt.Sprintf("fail to init registry.Registy, err:%s", jerrors.ErrorStack(err)))
-		return nil
-	}
-
-	// provider
-	srv = jsonrpc.NewServer(
-		jsonrpc.Registry(registry),
-		jsonrpc.ConfList(conf.ServerConfigList),
-		jsonrpc.ServiceConfList(conf.ServiceConfigList),
-	)
-
-	return srv
-}
-
-func uninitServer() {
-	if servo != nil {
-		servo.Stop()
-	}
-	log.Close()
-}
-
 func initProfiling() {
-	if !conf.Pprof_Enabled {
+	if !support.GetProviderConfig().Pprof_Enabled {
 		return
 	}
 	const (
@@ -109,7 +65,7 @@ func initProfiling() {
 	if err != nil {
 		panic("cat not get local ip!")
 	}
-	addr = ip + ":" + strconv.Itoa(conf.Pprof_Port)
+	addr = ip + ":" + strconv.Itoa(support.GetProviderConfig().Pprof_Port)
 	log.Info("App Profiling startup on address{%v}", addr+PprofPath)
 
 	go func() {
@@ -134,7 +90,6 @@ func initSignal() {
 			})
 
 			// 要么fastFailTimeout时间内执行完毕下面的逻辑然后程序退出,要么执行上面的超时函数程序强行退出
-			uninitServer()
 			fmt.Println("provider app exit now...")
 			return
 		}
diff --git a/examples/jsonrpc/go-server/app/user.go b/examples/jsonrpc/go-server/app/user.go
index d8f307549b3d7faab1c9404e01048f7cbd1b4a87..7ffe4260f50f553e33ac2ae2045d23b0809eabdb 100644
--- a/examples/jsonrpc/go-server/app/user.go
+++ b/examples/jsonrpc/go-server/app/user.go
@@ -4,6 +4,7 @@ import (
 	// "encoding/json"
 	"context"
 	"fmt"
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
 	"time"
 )
 
@@ -14,6 +15,10 @@ import (
 
 type Gender int
 
+func init() {
+	support.SetProService(new(UserProvider))
+}
+
 const (
 	MAN = iota
 	WOMAN
@@ -134,14 +139,14 @@ func (u *UserProvider) GetUser(ctx context.Context, req *string, rsp *User) erro
 }
 */
 
-func (u *UserProvider) GetUser(ctx context.Context, req []string, rsp *User) error {
+func (u *UserProvider) GetUser(ctx context.Context, req []interface{}, rsp *User) error {
 	var (
 		err  error
 		user *User
 	)
 
 	gxlog.CInfo("req:%#v", req)
-	user, err = u.getUser(req[0])
+	user, err = u.getUser(req[0].(string))
 	if err == nil {
 		*rsp = *user
 		gxlog.CInfo("rsp:%#v", rsp)
diff --git a/examples/jsonrpc/go-server/assembly/bin/load.sh b/examples/jsonrpc/go-server/assembly/bin/load.sh
index e202ff65f436f08191ae5364378f659de858777a..b1e7a976665722a525c9633c682f6e3e9a5e4cab 100644
--- a/examples/jsonrpc/go-server/assembly/bin/load.sh
+++ b/examples/jsonrpc/go-server/assembly/bin/load.sh
@@ -20,7 +20,7 @@ if [[ ${OS_NAME} != "Windows" ]]; then
     PROJECT_HOME=${PROJECT_HOME}"/"
 fi
 
-export APP_CONF_FILE=${PROJECT_HOME}"TARGET_CONF_FILE"
+export CONF_PROVIDER_FILE_PATH=${PROJECT_HOME}"TARGET_CONF_FILE"
 export APP_LOG_CONF_FILE=${PROJECT_HOME}"TARGET_LOG_CONF_FILE"
 
 usage() {
diff --git a/examples/jsonrpc/go-server/profiles/dev/server.yml b/examples/jsonrpc/go-server/profiles/dev/server.yml
index 2e0abd5151ca8fcc93375f227379e766d034aabf..b9e836fb679ce962835bf1ddf0178af0f9ff66e9 100644
--- a/examples/jsonrpc/go-server/profiles/dev/server.yml
+++ b/examples/jsonrpc/go-server/profiles/dev/server.yml
@@ -8,6 +8,7 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
     organization : "ikurento.com"
@@ -17,22 +18,42 @@ application_config:
     owner : "ZX"
     environment : "dev"
 
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        ip : "127.0.0.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "jsonrpc"
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "jsonrpc"
+    # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+#-   name: "dubbo"
+#    ip : "127.0.0.1"
+#    port : 20000
+-   name: "jsonrpc"
+    ip: "127.0.0.1"
+    port: 20001
+
diff --git a/examples/jsonrpc/go-server/profiles/release/server.yml b/examples/jsonrpc/go-server/profiles/release/server.yml
index ca33e45bccf08d09bc39338132ebfbfb13bd6fdc..0fb9d5d55d7f02dfb0ea86a0844e57deb43796a2 100644
--- a/examples/jsonrpc/go-server/profiles/release/server.yml
+++ b/examples/jsonrpc/go-server/profiles/release/server.yml
@@ -8,31 +8,52 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
-    organization : "ikurento.com"
-    name : "BDTService"
-    module : "dubbogo user-info server"
-    version : "0.0.1"
-    owner : "ZX"
-    environment : "product"
-
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        ip : "127.0.0.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "jsonrpc"
+  organization : "ikurento.com"
+  name : "BDTService"
+  module : "dubbogo user-info server"
+  version : "0.0.1"
+  owner : "ZX"
+  environment : "dev"
+
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "jsonrpc"
+  # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+#-   name: "dubbo"
+#    ip : "127.0.0.1"
+#    port : 20000
+-   name: "jsonrpc"
+    ip: "127.0.0.1"
+    port: 20001
+
diff --git a/examples/jsonrpc/go-server/profiles/test/server.yml b/examples/jsonrpc/go-server/profiles/test/server.yml
index 0c8be2383a37277713d7702d6b10aa366fedf2fd..0fb9d5d55d7f02dfb0ea86a0844e57deb43796a2 100644
--- a/examples/jsonrpc/go-server/profiles/test/server.yml
+++ b/examples/jsonrpc/go-server/profiles/test/server.yml
@@ -8,31 +8,52 @@ pprof_port : 20080
 transport : "http"
 net_timeout : "3s"
 
+path: ""
 # application config
 application_config:
-    organization : "ikurento.com"
-    name : "BDTService"
-    module : "dubbogo user-info server"
-    version : "0.0.1"
-    owner : "ZX"
-    environment : "test"
-
-registry: "zookeeper"
-
-zk_registry_config:
-    timeout	: "3s"
-    address:
-        - "127.0.0.1:2181"
-service_config_type: "default"
-service_list:
-    -
-        protocol : "jsonrpc"
-        # 相当于dubbo.xml中的interface
-        service : "com.ikurento.user.UserProvider"
-
-server_list:
-    -
-        ip : "127.0.0.1"
-        port : 20000
-        # 本server能够提供所有支持同样的Protocol的servicelist的服务
-        protocol : "jsonrpc"
+  organization : "ikurento.com"
+  name : "BDTService"
+  module : "dubbogo user-info server"
+  version : "0.0.1"
+  owner : "ZX"
+  environment : "dev"
+
+registries :
+- id: "hangzhouzk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2181"
+  username: ""
+  password: ""
+
+- id: "shanghaizk"
+  type: "zookeeper"
+  timeout	: "3s"
+  address: "127.0.0.1:2182"
+  username: ""
+  password: ""
+
+
+services:
+- registries:
+  - "hangzhouzk"
+  - "shanghaizk"
+  protocol : "jsonrpc"
+  # 相当于dubbo.xml中的interface
+  interface : "com.ikurento.user.UserProvider"
+  loadbalance: "random"
+  warmup: "100"
+  cluster: "failover"
+  methods:
+  - name: "GetUser"
+    retries: 1
+    loadbalance: "random"
+
+protocols:
+#-   name: "dubbo"
+#    ip : "127.0.0.1"
+#    port : 20000
+-   name: "jsonrpc"
+    ip: "127.0.0.1"
+    port: 20001
+
diff --git a/examples/jsonrpc/java-client/src/main/resources/dubbo.properties b/examples/jsonrpc/java-client/src/main/resources/dubbo.properties
deleted file mode 100644
index fc7b9aedd4260e82ab5f88327dca8728092c51b4..0000000000000000000000000000000000000000
--- a/examples/jsonrpc/java-client/src/main/resources/dubbo.properties
+++ /dev/null
@@ -1,13 +0,0 @@
-### dubboע���������� ###
-dubbo.container = log4j,spring
-dubbo.application.name = user-info-client
-dubbo.application.owner = AlexStocks
-dubbo.application.environment  =  product
-dubbo.registry.address = zookeeper://127.0.0.1:2181
-dubbo.monitor.protocol = zookeeper
-dubbo.consumer.timeout = 10000
-dubbo.provider.timeout = 10000
-dubbo.protocol.name = jsonrpc
-
-dubbo.log4j.file = logs/client.log
-dubbo.log4j.level = WARN
diff --git a/examples/readme.md b/examples/readme.md
deleted file mode 100644
index b6fbe59d50f3ecf9ce6759e57cdbfe7d9ae9a333..0000000000000000000000000000000000000000
--- a/examples/readme.md
+++ /dev/null
@@ -1,61 +0,0 @@
-## dubbogo examples
----
-    ***a golang apache dubbo code example. just using jsonrpc 2.0 protocol over http now.***
-
-### Introduction
-
-examples是基于dubbogo的实现的代码示例,目前提供user-info一个例子。examples/client 是一个 go client,借鉴java的编译思路,提供了区别于一般的go程序的而类似于java的独特的编译脚本系统。
-
-### Code Example
-
-#### 1: user-info 
-
-*从这个程序可以看出dubbogo client程序(user-info/client) 如何调用 java(dubbo) server的程序( user-info/java-server)的。*
-
-**Attention**: 测试的时候一定注意修改配置文件中服务端和zk的ip&port。
-
-+ 1 部署zookeeper服务;
-+ 2 请编译并部署 examples/java-server,注意修改zk地址(conf/dubbo.properties:line6:"dubbo.registry.address")和监听端口(conf/dubbo.properties:line6:"dubbo.protocol.port", 不建议修改port), 然后执行"bin/start.sh"启动java服务端;
-+ 3 修改 examples/client/profiles/test/client.toml:line 31,写入正确的zk地址;
-+ 4 examples/client/ 下执行 sh assembly/mac/dev.sh命令(linux下请执行sh assembly/linux/dev.sh)
-
-	target/darwin 下即放置好了编译好的程序以及打包结果,在 client/target/darwin/user_info_client-0.2.0-20180808-1258-dev 下执行sh bin/load.sh start命令即可客户端程序;
-
-### QA List
-
-#### 1 Unsupported protocol jsonrpc
-
-问题描述: 用java写的dubbo provider端程序如果遇到下面的错误:
-
-	 java.lang.IllegalStateException: Unsupported protocol jsonrpc in notified url:   
-	
-	 jsonrpc://116.211.15.189:8080/im.ikurento.user.UserProvider?anyhost=true&application=token-dubbo-p&default.threads=100&dubbo=2.5.3…
-	
-	 from registry 116.211.15.190:2181 to consumer 10.241.19.54, supported protocol: [dubbo, http, injvm, mock, redis, registry, rmi, thrift]
-
-错误原因:provider端声明使用了jsonrpc,所以所有的协议都默认支持了jsonrpc协议。
-
-解决方法:服务需要在dubbo.provider.xml中明确支持dubbo协议,请在reference中添加protocol="dubbo",如:
-
-    <dubbo:protocol name="dubbo" port="28881" />
-    <dubbo:protocol name="jsonrpc" port="38881" server="jetty" />
-    <dubbo:service id="userService" interface="im.ikurento.user.UserProvider" check="false" timeout="5000" protocol="dubbo"/>
-
- 	<dubbo:service id="userService" interface="im.ikurento.user.UserProvider" check="false" timeout="5000" protocol="jsonrpc"/>
-
-与本问题无关,补充一些消费端的配置:
-
-    <dubbo:reference id="userService" interface="im.ikurento.user.UserService" connections="2" check="false">
-    	<dubbo:method name="GetUser" async="true" return="false" />
- 	</dubbo:reference>
-
-#### 2 配置文件
-
-dubbogo client端根据其配置文件client.toml的serviceList来watch zookeeper上相关服务,否则当相关服务的zk provider path下的node发生增删的时候,因为关注的service不正确而导致不能收到相应的通知。
-
-所以务必注意这个配置文件中的serviceList与实际代码中相关类的Service函数提供的Service一致。
-
-#### 3 其他注意事项
-- a. dubbo 可以配置多个 zk 地址;
-- b. 消费者在 dubbo 的配置文件中相关interface配置务必指定protocol, 如protocol="dubbo";
-- c. java dubbo provider提供服务的method如果要提供给dubbogo client服务,则method的首字母必须大写;
\ No newline at end of file
diff --git a/filter/filter.go b/filter/filter.go
new file mode 100644
index 0000000000000000000000000000000000000000..6108040bb14ca1a7f1570b54ab357dc4165036e4
--- /dev/null
+++ b/filter/filter.go
@@ -0,0 +1,11 @@
+package filter
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+// Extension - Filter
+type Filter interface {
+	Invoke(protocol.Invoker, protocol.Invocation) protocol.Result
+	OnResponse(protocol.Result, protocol.Invoker, protocol.Invocation) protocol.Result
+}
diff --git a/filter/imp/echo_filter.go b/filter/imp/echo_filter.go
new file mode 100644
index 0000000000000000000000000000000000000000..13631f7a2d1dc6bf707a47d931c606e37cb6c73c
--- /dev/null
+++ b/filter/imp/echo_filter.go
@@ -0,0 +1,42 @@
+package imp
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/filter"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+const ECHO = "echo"
+
+func init() {
+	extension.SetFilter(ECHO, GetFilter)
+}
+
+// RPCService need a Echo method in consumer, if you want to use EchoFilter
+// eg:
+//		Echo func(ctx context.Context, args []interface{}, rsp *Xxx) error
+type EchoFilter struct {
+}
+
+func (ef *EchoFilter) Invoke(invoker protocol.Invoker, invocation protocol.Invocation) protocol.Result {
+	log.Info("invoking echo filter.")
+	if invocation.MethodName() == constant.ECHO && len(invocation.Arguments()) == 1 {
+		return &protocol.RPCResult{
+			Rest: invocation.Arguments()[0],
+		}
+	}
+	return invoker.Invoke(invocation)
+}
+
+func (ef *EchoFilter) OnResponse(result protocol.Result, invoker protocol.Invoker, invocation protocol.Invocation) protocol.Result {
+	return result
+}
+
+func GetFilter() filter.Filter {
+	return &EchoFilter{}
+}
diff --git a/go.mod b/go.mod
index 8abc40265af71ddb6db8a122d6346edfc3f4f417..02336cbdce5c8b265b490c5170a011de91c14655 100644
--- a/go.mod
+++ b/go.mod
@@ -1,4 +1,4 @@
-module github.com/dubbo/dubbo-go
+module github.com/dubbo/go-for-apache-dubbo
 
 require (
 	github.com/AlexStocks/getty v0.0.0-20190331201845-1ca64ac5a589
@@ -7,5 +7,8 @@ require (
 	github.com/dubbogo/hessian2 v0.0.0-20190410112310-f093e4436e31
 	github.com/juju/errors v0.0.0-20190207033735-e65537c515d7
 	github.com/samuel/go-zookeeper v0.0.0-20180130194729-c4fab1ac1bec
+	github.com/stretchr/testify v1.3.0
+	github.com/tevino/abool v0.0.0-20170917061928-9b9efcf221b5
+	go.uber.org/atomic v1.3.2
 	gopkg.in/yaml.v2 v2.2.2
 )
diff --git a/go.sum b/go.sum
index c4f8086db8b059460c5a0565dbf57de62d8d2391..99fd3c20afc707e23c9cf389214327b7f033543a 100644
--- a/go.sum
+++ b/go.sum
@@ -8,6 +8,7 @@ github.com/AlexStocks/goext v0.3.2/go.mod h1:3M5j9Pjge4CdkNg2WIjRLUeoPedJHHKwkkg
 github.com/AlexStocks/log4go v1.0.2 h1:1K5WM8KjSUECaoXUl8FSF05KGeCJDfBrhKUBsxwUvhk=
 github.com/AlexStocks/log4go v1.0.2/go.mod h1:6kCCRo/orDo8mh5CEDOeuSSM674wBQ8M6E0K8dVOIz4=
 github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU=
+github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0=
 github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
 github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927/go.mod h1:h/aW8ynjgkuj+NQRlZcDbAbM1ORAbXjXX77sX7T289U=
 github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw=
@@ -32,6 +33,7 @@ github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfU
 github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc=
 github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A=
 github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
+github.com/golang/protobuf v1.3.1 h1:YF8+flBXS5eO826T4nzqPrxfhQThhXl0YzfuUPu4SBg=
 github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U=
 github.com/golang/snappy v0.0.1 h1:Qgr9rKW7uDUkrbSmQeiDsGa8SjGyCOGtuasMWwvp2P4=
 github.com/golang/snappy v0.0.1/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q=
@@ -53,6 +55,7 @@ github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073 h1:WQM1NildKThwdP7qWr
 github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073 h1:WQM1NildKThwdP7qWrNAFGzp4ijNLw8RlgENkaI4MJs=
 github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA=
 github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA=
+github.com/juju/utils v0.0.0-20180820210520-bf9cc5bdd62d h1:irPlN9z5VCe6BTsqVsxheCZH99OFSmqSVyTigW4mEoY=
 github.com/juju/utils v0.0.0-20180820210520-bf9cc5bdd62d/go.mod h1:6/KLg8Wz/y2KVGWEpkK9vMNGkOnu4k/cqs8Z1fKjTOk=
 github.com/juju/version v0.0.0-20180108022336-b64dbd566305/go.mod h1:kE8gK5X0CImdr7qpSKl3xB2PmpySSmfj7zVbkZFs81U=
 github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88 h1:uC1QfSlInpQF+M0ao65imhwqKnz3Q2z/d8PWZRMQvDM=
@@ -61,6 +64,7 @@ github.com/k0kubun/pp v3.0.0+incompatible h1:I00lq/ALERE8g5dW2th1jnjtJ/J4vautUNy
 github.com/k0kubun/pp v3.0.0+incompatible/go.mod h1:GWse8YhT0p8pT4ir3ZgBbfZild3tgzSScAn6HmfYukg=
 github.com/kisielk/errcheck v1.1.0/go.mod h1:EZBBE59ingxPouuu3KfxchcWSUPOHkagtvWXihfKN4Q=
 github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck=
+github.com/konsorten/go-windows-terminal-sequences v1.0.1 h1:mweAR1A6xJ3oS2pRaGiHgQ4OO8tzTaLawm8vnODuwDk=
 github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ=
 github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI=
 github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo=
@@ -74,6 +78,7 @@ github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcncea
 github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
 github.com/mattn/go-isatty v0.0.7 h1:UvyT9uN+3r7yLEYSlJsbQGdsaB/a0DlgWP3pql6iwOc=
 github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s=
+github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU=
 github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0=
 github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q=
 github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0=
@@ -86,16 +91,21 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb
 github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM=
 github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
 github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4=
+github.com/prometheus/client_golang v0.9.2 h1:awm861/B8OKDd2I/6o1dy3ra4BamzKhYOiGItCeZ740=
 github.com/prometheus/client_golang v0.9.2/go.mod h1:OsXs2jCmiKlQ1lTBmv21f2mNfw4xf/QclQDMrYNZzcM=
 github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo=
+github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90 h1:S/YWwWx/RA8rT8tKFRuGUZhuA90OyIBpPCXkcbwU8DE=
 github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA=
+github.com/prometheus/common v0.0.0-20181126121408-4724e9255275 h1:PnBWHBf+6L0jOqq0gIVUe6Yk0/QMZ640k6NvkxcBf+8=
 github.com/prometheus/common v0.0.0-20181126121408-4724e9255275/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro=
+github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a h1:9a8MnZMP0X2nLJdBg+pBmGgkJlSaKC2KaQmTCk1XDtE=
 github.com/prometheus/procfs v0.0.0-20181204211112-1dc9a6cbc91a/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk=
 github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg=
 github.com/samuel/go-zookeeper v0.0.0-20180130194729-c4fab1ac1bec h1:6ncX5ko6B9LntYM0YBRXkiSaZMmLYeZ/NWcmeB43mMY=
 github.com/samuel/go-zookeeper v0.0.0-20180130194729-c4fab1ac1bec h1:6ncX5ko6B9LntYM0YBRXkiSaZMmLYeZ/NWcmeB43mMY=
 github.com/samuel/go-zookeeper v0.0.0-20180130194729-c4fab1ac1bec/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
 github.com/samuel/go-zookeeper v0.0.0-20180130194729-c4fab1ac1bec/go.mod h1:gi+0XIa01GRL2eRQVjQkKGqKF3SF9vZR/HnPullcV2E=
+github.com/sirupsen/logrus v1.4.0 h1:yKenngtzGh+cUSSh6GWbxW2abRqhYUSR/t/6+2QqNvE=
 github.com/sirupsen/logrus v1.4.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo=
 github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM=
 github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME=
@@ -106,15 +116,19 @@ github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0
 github.com/stretchr/testify v1.3.0 h1:TivCn/peBQ7UY8ooIcPgZFpTNSz0Q2U6UrFlUfqbe0Q=
 github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
 github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI=
+github.com/tevino/abool v0.0.0-20170917061928-9b9efcf221b5 h1:hNna6Fi0eP1f2sMBe/rJicDmaHmoXGe1Ta84FPYHLuE=
+github.com/tevino/abool v0.0.0-20170917061928-9b9efcf221b5/go.mod h1:f1SCnEOt6sc3fOJfPQDRDzHOtSXuTtnz0ImG9kPRDV0=
 github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U=
 github.com/ugorji/go v1.1.2/go.mod h1:hnLbHMwcvSihnDhEfx2/BzKp2xb0Y+ErdfYcrs9tkJQ=
 github.com/ugorji/go/codec v0.0.0-20190320090025-2dc34c0b8780/go.mod h1:iT03XoTwV7xq/+UGwKO3UbC1nNNlopQiY61beSdrtOA=
 github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU=
 go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU=
+go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4=
 go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE=
 go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=
 go.uber.org/zap v1.9.1/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q=
 golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4=
+golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M=
 golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w=
 golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE=
 golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4=
diff --git a/jsonrpc/map.go b/jsonrpc/map.go
deleted file mode 100644
index 43c083330d082a6e75d3501663fc81bbac6b26ec..0000000000000000000000000000000000000000
--- a/jsonrpc/map.go
+++ /dev/null
@@ -1,285 +0,0 @@
-package jsonrpc
-
-import (
-	"bytes"
-	"context"
-	"io"
-	"io/ioutil"
-	"net"
-	"net/http"
-	"reflect"
-	"strings"
-	"sync"
-	"unicode"
-	"unicode/utf8"
-)
-
-import (
-	log "github.com/AlexStocks/log4go"
-	jerrors "github.com/juju/errors"
-)
-
-var (
-	// A value sent as a placeholder for the server's response value when the server
-	// receives an invalid request. It is never decoded by the client since the Response
-	// contains an error when it is used.
-	invalidRequest = struct{}{}
-
-	// Precompute the reflect type for error. Can't use error directly
-	// because Typeof takes an empty interface value. This is annoying.
-	typeOfError = reflect.TypeOf((*error)(nil)).Elem()
-)
-
-type serviceMethod struct {
-	method    reflect.Method // receiver method
-	ctxType   reflect.Type   // type of the request context
-	argsType  reflect.Type   // type of the request argument
-	replyType reflect.Type   // type of the response argument
-}
-
-func (m *serviceMethod) suiteContext(ctx context.Context) reflect.Value {
-	if contextv := reflect.ValueOf(ctx); contextv.IsValid() {
-		return contextv
-	}
-	return reflect.Zero(m.ctxType)
-}
-
-type svc struct {
-	name     string                    // name of service
-	rcvr     reflect.Value             // receiver of methods for the service
-	rcvrType reflect.Type              // type of the receiver
-	methods  map[string]*serviceMethod // registered methods, function name -> reflect.function
-}
-
-type serviceMap struct {
-	mutex      sync.Mutex      // protects the serviceMap
-	serviceMap map[string]*svc // service name -> service
-}
-
-func initServer() *serviceMap {
-	return &serviceMap{
-		serviceMap: make(map[string]*svc),
-	}
-}
-
-// isExported returns true of a string is an exported (upper case) name.
-func isExported(name string) bool {
-	rune, _ := utf8.DecodeRuneInString(name)
-	return unicode.IsUpper(rune)
-}
-
-// isExportedOrBuiltin returns true if a type is exported or a builtin.
-func isExportedOrBuiltin(t reflect.Type) bool {
-	for t.Kind() == reflect.Ptr {
-		t = t.Elem()
-	}
-	// PkgPath will be non-empty even for an exported type,
-	// so we need to check the type name as well.
-	return isExported(t.Name()) || t.PkgPath() == ""
-}
-
-func suiteMethod(method reflect.Method) *serviceMethod {
-	mtype := method.Type
-	mname := method.Name
-
-	// Method must be exported.
-	if method.PkgPath != "" {
-		return nil
-	}
-
-	var replyType, argType, ctxType reflect.Type
-	switch mtype.NumIn() {
-	case 3:
-		argType = mtype.In(1)
-		replyType = mtype.In(2)
-	case 4:
-		ctxType = mtype.In(1)
-		argType = mtype.In(2)
-		replyType = mtype.In(3)
-	default:
-		log.Error("method %s of mtype %v has wrong number of in parameters %d; needs exactly 3/4",
-			mname, mtype, mtype.NumIn())
-		return nil
-	}
-	// First arg need not be a pointer.
-	if !isExportedOrBuiltin(argType) {
-		log.Error("argument type of method %q is not exported %v", mname, argType)
-		return nil
-	}
-	// Second arg must be a pointer.
-	if replyType.Kind() != reflect.Ptr {
-		log.Error("reply type of method %q is not a pointer %v", mname, replyType)
-		return nil
-	}
-	// Reply type must be exported.
-	if !isExportedOrBuiltin(replyType) {
-		log.Error("reply type of method %s not exported{%v}", mname, replyType)
-		return nil
-	}
-	// Method needs one out.
-	if mtype.NumOut() != 1 {
-		log.Error("method %q has %d out parameters; needs exactly 1", mname, mtype.NumOut())
-		return nil
-	}
-	// The return type of the method must be error.
-	if returnType := mtype.Out(0); returnType != typeOfError {
-		log.Error("return type %s of method %q is not error", returnType, mname)
-		return nil
-	}
-
-	return &serviceMethod{method: method, argsType: argType, replyType: replyType, ctxType: ctxType}
-}
-
-func (server *serviceMap) register(rcvr Handler) (string, error) {
-	server.mutex.Lock()
-	defer server.mutex.Unlock()
-	if server.serviceMap == nil {
-		server.serviceMap = make(map[string]*svc)
-	}
-
-	s := new(svc)
-	s.rcvrType = reflect.TypeOf(rcvr)
-	s.rcvr = reflect.ValueOf(rcvr)
-	sname := reflect.Indirect(s.rcvr).Type().Name()
-	if sname == "" {
-		s := "no service name for type " + s.rcvrType.String()
-		log.Error(s)
-		return "", jerrors.New(s)
-	}
-	if !isExported(sname) {
-		s := "type " + sname + " is not exported"
-		log.Error(s)
-		return "", jerrors.New(s)
-	}
-
-	sname = rcvr.Service()
-	if _, dup := server.serviceMap[sname]; dup {
-		return "", jerrors.New("service already defined: " + sname)
-	}
-	s.name = sname
-	s.methods = make(map[string]*serviceMethod)
-
-	// Install the methods
-	methods := ""
-	num := s.rcvrType.NumMethod()
-	for m := 0; m < num; m++ {
-		method := s.rcvrType.Method(m)
-		if mt := suiteMethod(method); mt != nil {
-			s.methods[method.Name] = mt
-			methods += method.Name + ","
-		}
-	}
-
-	if len(s.methods) == 0 {
-		s := "type " + sname + " has no exported methods of suitable type"
-		log.Error(s)
-		return "", jerrors.New(s)
-	}
-	server.serviceMap[s.name] = s
-
-	return strings.TrimSuffix(methods, ","), nil
-}
-
-func (server *serviceMap) serveRequest(ctx context.Context,
-	header map[string]string, body []byte, conn net.Conn) error {
-
-	// read request header
-	codec := newServerCodec()
-	err := codec.ReadHeader(header, body)
-	if err != nil {
-		if err == io.EOF || err == io.ErrUnexpectedEOF {
-			return jerrors.Trace(err)
-		}
-
-		return jerrors.New("server cannot decode request: " + err.Error())
-	}
-	serviceName := header["Path"]
-	methodName := codec.req.Method
-	if len(serviceName) == 0 || len(methodName) == 0 {
-		codec.ReadBody(nil)
-		return jerrors.New("service/method request ill-formed: " + serviceName + "/" + methodName)
-	}
-
-	// get method
-	server.mutex.Lock()
-	svc := server.serviceMap[serviceName]
-	server.mutex.Unlock()
-	if svc == nil {
-		codec.ReadBody(nil)
-		return jerrors.New("cannot find svc " + serviceName)
-	}
-	mtype := svc.methods[methodName]
-	if mtype == nil {
-		codec.ReadBody(nil)
-		return jerrors.New("cannot find method " + methodName + " of svc " + serviceName)
-	}
-
-	// get args
-	var argv reflect.Value
-	argIsValue := false
-	if mtype.argsType.Kind() == reflect.Ptr {
-		argv = reflect.New(mtype.argsType.Elem())
-	} else {
-		argv = reflect.New(mtype.argsType)
-		argIsValue = true
-	}
-	// argv guaranteed to be a pointer now.
-	if err = codec.ReadBody(argv.Interface()); err != nil {
-		return jerrors.Trace(err)
-	}
-	if argIsValue {
-		argv = argv.Elem()
-	}
-
-	replyv := reflect.New(mtype.replyType.Elem())
-
-	//  call service.method(args)
-	var errMsg string
-	returnValues := mtype.method.Func.Call([]reflect.Value{
-		svc.rcvr,
-		mtype.suiteContext(ctx),
-		reflect.ValueOf(argv.Interface()),
-		reflect.ValueOf(replyv.Interface()),
-	})
-	// The return value for the method is an error.
-	if retErr := returnValues[0].Interface(); retErr != nil {
-		errMsg = retErr.(error).Error()
-	}
-
-	// write response
-	code := 200
-	rspReply := replyv.Interface()
-	if len(errMsg) != 0 {
-		code = 500
-		rspReply = invalidRequest
-	}
-	rspStream, err := codec.Write(errMsg, rspReply)
-	if err != nil {
-		return jerrors.Trace(err)
-	}
-	rsp := &http.Response{
-		StatusCode:    code,
-		ProtoMajor:    1,
-		ProtoMinor:    1,
-		Header:        make(http.Header),
-		ContentLength: int64(len(rspStream)),
-		Body:          ioutil.NopCloser(bytes.NewReader(rspStream)),
-	}
-	delete(header, "Content-Type")
-	delete(header, "Content-Length")
-	delete(header, "Timeout")
-	for k, v := range header {
-		rsp.Header.Set(k, v)
-	}
-
-	rspBuf := bytes.NewBuffer(make([]byte, DefaultHTTPRspBufferSize))
-	rspBuf.Reset()
-	if err = rsp.Write(rspBuf); err != nil {
-		log.Warn("rsp.Write(rsp:%#v) = error:%s", rsp, err)
-		return nil
-	}
-	if _, err = rspBuf.WriteTo(conn); err != nil {
-		log.Warn("rspBuf.WriteTo(conn:%#v) = error:%s", conn, err)
-	}
-	return nil
-}
diff --git a/jsonrpc/server.go b/jsonrpc/server.go
deleted file mode 100644
index d8604d4b17895b25da3383e0deea86c1fa692430..0000000000000000000000000000000000000000
--- a/jsonrpc/server.go
+++ /dev/null
@@ -1,383 +0,0 @@
-package jsonrpc
-
-import (
-	"bufio"
-	"bytes"
-	"context"
-	"github.com/dubbo/dubbo-go/plugins"
-	"io/ioutil"
-	"net"
-	"net/http"
-	"runtime"
-	"runtime/debug"
-	"sync"
-	"time"
-)
-
-import (
-	"github.com/AlexStocks/goext/net"
-	log "github.com/AlexStocks/log4go"
-	"github.com/dubbo/dubbo-go/registry"
-	jerrors "github.com/juju/errors"
-)
-
-import (
-	"github.com/dubbo/dubbo-go/server"
-)
-
-const (
-	DefaultMaxSleepTime      = 1 * time.Second // accept中间最大sleep interval
-	DefaultHTTPRspBufferSize = 1024
-	PathPrefix               = byte('/')
-)
-
-// Handler interface represents a Service request handler. It's generated
-// by passing any type of public concrete object with methods into server.NewHandler.
-// Most will pass in a struct.
-//
-// Example:
-//
-//	type Hello struct {}
-//
-//	func (s *Hello) Method(context, request, response) error {
-//		return nil
-//	}
-//
-//  func (s *Hello) Service() string {
-//      return "com.youni.service"
-//  }
-//
-//  func (s *Hello) Version() string {
-//      return "1.0.0"
-//  }
-
-type Handler interface {
-	Service() string // Service Interface
-	Version() string
-}
-
-type Option func(*Options)
-
-type Options struct {
-	Registry        registry.Registry
-	ConfList        []server.ServerConfig
-	ServiceConfList []registry.ProviderServiceConfig
-	Timeout         time.Duration
-}
-
-func newOptions(opt ...Option) Options {
-	opts := Options{}
-	for _, o := range opt {
-		o(&opts)
-	}
-
-	if opts.Registry == nil {
-		panic("server.Options.Registry is nil")
-	}
-
-	return opts
-}
-
-// Registry used for discovery
-func Registry(r registry.Registry) Option {
-	return func(o *Options) {
-		o.Registry = r
-	}
-}
-
-func ConfList(confList []server.ServerConfig) Option {
-	return func(o *Options) {
-		o.ConfList = confList
-		for i := 0; i < len(o.ConfList); i++ {
-			if o.ConfList[i].IP == "" {
-				o.ConfList[i].IP, _ = gxnet.GetLocalIP()
-			}
-		}
-	}
-}
-
-func ServiceConfList(confList []registry.ProviderServiceConfig) Option {
-	return func(o *Options) {
-		o.ServiceConfList = confList
-	}
-}
-
-type Server struct {
-	rpc  []*serviceMap
-	done chan struct{}
-	once sync.Once
-
-	sync.RWMutex
-	opts     Options
-	handlers map[string]Handler
-	wg       sync.WaitGroup
-}
-
-func NewServer(opts ...Option) *Server {
-	var (
-		num int
-	)
-	options := newOptions(opts...)
-	Servers := make([]*serviceMap, len(options.ConfList))
-	num = len(options.ConfList)
-	for i := 0; i < num; i++ {
-		Servers[i] = initServer()
-	}
-	return &Server{
-		opts:     options,
-		rpc:      Servers,
-		handlers: make(map[string]Handler),
-		done:     make(chan struct{}),
-	}
-}
-
-func (s *Server) handlePkg(rpc *serviceMap, conn net.Conn) {
-	defer func() {
-		if r := recover(); r != nil {
-			log.Warn("connection{local:%v, remote:%v} panic error:%#v, debug stack:%s",
-				conn.LocalAddr(), conn.RemoteAddr(), r, string(debug.Stack()))
-		}
-
-		conn.Close()
-	}()
-
-	setReadTimeout := func(conn net.Conn, timeout time.Duration) {
-		t := time.Time{}
-		if timeout > time.Duration(0) {
-			t = time.Now().Add(timeout)
-		}
-
-		conn.SetDeadline(t)
-	}
-
-	sendErrorResp := func(header http.Header, body []byte) error {
-		rsp := &http.Response{
-			Header:        header,
-			StatusCode:    500,
-			ContentLength: int64(len(body)),
-			Body:          ioutil.NopCloser(bytes.NewReader(body)),
-		}
-		rsp.Header.Del("Content-Type")
-		rsp.Header.Del("Content-Length")
-		rsp.Header.Del("Timeout")
-
-		rspBuf := bytes.NewBuffer(make([]byte, DefaultHTTPRspBufferSize))
-		rspBuf.Reset()
-		err := rsp.Write(rspBuf)
-		if err != nil {
-			return jerrors.Trace(err)
-		}
-		_, err = rspBuf.WriteTo(conn)
-		return jerrors.Trace(err)
-	}
-
-	for {
-		bufReader := bufio.NewReader(conn)
-		r, err := http.ReadRequest(bufReader)
-		if err != nil {
-			return
-		}
-
-		reqBody, err := ioutil.ReadAll(r.Body)
-		if err != nil {
-			return
-		}
-		r.Body.Close()
-
-		reqHeader := make(map[string]string)
-		for k := range r.Header {
-			reqHeader[k] = r.Header.Get(k)
-		}
-		reqHeader["Path"] = r.URL.Path[1:] // to get service name
-		if r.URL.Path[0] != PathPrefix {
-			reqHeader["Path"] = r.URL.Path
-		}
-		reqHeader["HttpMethod"] = r.Method
-
-		httpTimeout := s.Options().Timeout
-		contentType := reqHeader["Content-Type"]
-		if contentType != "application/json" && contentType != "application/json-rpc" {
-			setReadTimeout(conn, httpTimeout)
-			r.Header.Set("Content-Type", "text/plain")
-			if errRsp := sendErrorResp(r.Header, []byte(jerrors.ErrorStack(err))); errRsp != nil {
-				log.Warn("sendErrorResp(header:%#v, error:%s) = error:%s",
-					r.Header, jerrors.ErrorStack(err), errRsp)
-			}
-			return
-		}
-
-		ctx := context.Background()
-		if len(reqHeader["Timeout"]) > 0 {
-			timeout, err := time.ParseDuration(reqHeader["Timeout"])
-			if err == nil {
-				httpTimeout = timeout
-				ctx, _ = context.WithTimeout(ctx, httpTimeout)
-			}
-			delete(reqHeader, "Timeout")
-		}
-		setReadTimeout(conn, httpTimeout)
-
-		if err := rpc.serveRequest(ctx, reqHeader, reqBody, conn); err != nil {
-			if errRsp := sendErrorResp(r.Header, []byte(jerrors.ErrorStack(err))); errRsp != nil {
-				log.Warn("sendErrorResp(header:%#v, error:%s) = error:%s",
-					r.Header, jerrors.ErrorStack(err), errRsp)
-			}
-
-			log.Info("Unexpected error serving request, closing socket: %v", err)
-			return
-		}
-	}
-}
-
-func (s *Server) Options() Options {
-	s.RLock()
-	defer s.RUnlock()
-	return s.opts
-}
-
-func (s *Server) Handle(h Handler) error {
-	var (
-		err error
-	)
-
-	opts := s.Options()
-	serviceConf := plugins.DefaultProviderServiceConfig()()
-
-	serviceConf.SetService(h.Service())
-	serviceConf.SetVersion(h.Version())
-
-	flag := 0
-	serviceNum := len(opts.ServiceConfList)
-	ServerNum := len(opts.ConfList)
-	for i := 0; i < serviceNum; i++ {
-		if opts.ServiceConfList[i].Service() == serviceConf.Service() &&
-			opts.ServiceConfList[i].Version() == serviceConf.Version() {
-
-			serviceConf.SetProtocol(opts.ServiceConfList[i].Protocol())
-			serviceConf.SetGroup(opts.ServiceConfList[i].Group())
-			// serviceConf.Version = opts.ServiceConfList[i].Version
-			var methods, path string
-			for j := 0; j < ServerNum; j++ {
-				if opts.ConfList[j].Protocol == serviceConf.Protocol() {
-					s.Lock()
-					methods, err = s.rpc[j].register(h)
-					s.Unlock()
-					if err != nil {
-						return err
-					}
-					serviceConf.SetMethods(methods)
-
-					path = opts.ConfList[j].Address()
-					serviceConf.SetPath(path)
-					err = opts.Registry.Register(serviceConf)
-					if err != nil {
-						return err
-					}
-					flag = 1
-				}
-			}
-		}
-	}
-
-	if flag == 0 {
-		return jerrors.Errorf("fail to register Handler{service:%s, version:%s}",
-			serviceConf.Service(), serviceConf.Version())
-	}
-
-	s.Lock()
-	s.handlers[h.Service()] = h
-	s.Unlock()
-
-	return nil
-}
-
-func accept(listener net.Listener, fn func(net.Conn)) error {
-	var (
-		err      error
-		c        net.Conn
-		ok       bool
-		ne       net.Error
-		tmpDelay time.Duration
-	)
-
-	for {
-		c, err = listener.Accept()
-		if err != nil {
-			if ne, ok = err.(net.Error); ok && ne.Temporary() {
-				if tmpDelay != 0 {
-					tmpDelay <<= 1
-				} else {
-					tmpDelay = 5 * time.Millisecond
-				}
-				if tmpDelay > DefaultMaxSleepTime {
-					tmpDelay = DefaultMaxSleepTime
-				}
-				log.Info("http: Accept error: %v; retrying in %v\n", err, tmpDelay)
-				time.Sleep(tmpDelay)
-				continue
-			}
-			return jerrors.Trace(err)
-		}
-
-		go func() {
-			defer func() {
-				if r := recover(); r != nil {
-					const size = 64 << 10
-					buf := make([]byte, size)
-					buf = buf[:runtime.Stack(buf, false)]
-					log.Error("http: panic serving %v: %v\n%s", c.RemoteAddr(), r, buf)
-					c.Close()
-				}
-			}()
-
-			fn(c)
-		}()
-	}
-}
-
-func (s *Server) Start() error {
-	config := s.Options()
-
-	ServerNum := len(config.ConfList)
-	for i := 0; i < ServerNum; i++ {
-		listener, err := net.Listen("tcp", config.ConfList[i].Address())
-		if err != nil {
-			return err
-		}
-		log.Info("rpc server start to listen on %s", listener.Addr())
-
-		s.Lock()
-		rpc := s.rpc[i]
-		s.Unlock()
-
-		s.wg.Add(1)
-		go func(servo *serviceMap) {
-			accept(listener, func(conn net.Conn) { s.handlePkg(rpc, conn) })
-			s.wg.Done()
-		}(rpc)
-
-		s.wg.Add(1)
-		go func(servo *serviceMap) { // Server done goroutine
-			var err error
-			<-s.done               // step1: block to wait for done channel(wait Server.Stop step2)
-			err = listener.Close() // step2: and then close listener
-			if err != nil {
-				log.Warn("listener{addr:%s}.Close() = error{%#v}", listener.Addr(), err)
-			}
-			s.wg.Done()
-		}(rpc)
-	}
-
-	return nil
-}
-
-func (s *Server) Stop() {
-	s.once.Do(func() {
-		close(s.done)
-		s.wg.Wait()
-		if s.opts.Registry != nil {
-			s.opts.Registry.Close()
-			s.opts.Registry = nil
-		}
-	})
-}
diff --git a/plugins/plugins.go b/plugins/plugins.go
deleted file mode 100644
index 9648420d74d931ac51afe636f40eb6caf497b0cc..0000000000000000000000000000000000000000
--- a/plugins/plugins.go
+++ /dev/null
@@ -1,53 +0,0 @@
-package plugins
-
-import (
-	"github.com/dubbo/dubbo-go/client/selector"
-	"github.com/dubbo/dubbo-go/registry"
-)
-
-var PluggableRegistries = map[string]func(...registry.RegistryOption) (registry.Registry, error){}
-
-var PluggableLoadbalance = map[string]func() selector.Selector{
-	"round_robin": selector.NewRoundRobinSelector,
-	"random":      selector.NewRandomSelector,
-}
-
-// service configuration plugins , related to SeviceConfig for consumer paramters / ProviderSeviceConfig for provider parameters /
-
-// TODO:ServiceEven & ServiceURL subscribed by consumer from provider's listener shoud abstract to interface
-var PluggableServiceConfig = map[string]func() registry.ServiceConfig{
-	"default": registry.NewDefaultServiceConfig,
-}
-var PluggableProviderServiceConfig = map[string]func() registry.ProviderServiceConfig{
-	"default": registry.NewDefaultProviderServiceConfig,
-}
-
-var PluggableServiceURL = map[string]func(string) (registry.ServiceURL, error){
-	"default": registry.NewDefaultServiceURL,
-}
-
-var defaultServiceConfig = registry.NewDefaultServiceConfig
-var defaultProviderServiceConfig = registry.NewDefaultProviderServiceConfig
-
-var defaultServiceURL = registry.NewDefaultServiceURL
-
-func SetDefaultServiceConfig(s string) {
-	defaultServiceConfig = PluggableServiceConfig[s]
-}
-func DefaultServiceConfig() func() registry.ServiceConfig {
-	return defaultServiceConfig
-}
-
-func SetDefaultProviderServiceConfig(s string) {
-	defaultProviderServiceConfig = PluggableProviderServiceConfig[s]
-}
-func DefaultProviderServiceConfig() func() registry.ProviderServiceConfig {
-	return defaultProviderServiceConfig
-}
-
-func SetDefaultServiceURL(s string) {
-	defaultServiceURL = PluggableServiceURL[s]
-}
-func DefaultServiceURL() func(string) (registry.ServiceURL, error) {
-	return defaultServiceURL
-}
diff --git a/dubbo/client.go b/protocol/dubbo/client.go
similarity index 75%
rename from dubbo/client.go
rename to protocol/dubbo/client.go
index 9ea94e8b393b2133e6f89735efa787dc7de5e983..60a1f25e0b3a171d969945f928ea5f4b6f6ee6de 100644
--- a/dubbo/client.go
+++ b/protocol/dubbo/client.go
@@ -1,7 +1,8 @@
 package dubbo
 
 import (
-	"math/rand"
+	"io/ioutil"
+	"os"
 	"strings"
 	"sync"
 	"time"
@@ -10,13 +11,15 @@ import (
 import (
 	"github.com/AlexStocks/getty"
 	"github.com/AlexStocks/goext/sync/atomic"
+	log "github.com/AlexStocks/log4go"
 	"github.com/dubbogo/hessian2"
 	jerrors "github.com/juju/errors"
+	"gopkg.in/yaml.v2"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/public"
-	"github.com/dubbo/dubbo-go/registry"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
 )
 
 var (
@@ -25,10 +28,49 @@ var (
 	errSessionNotExist   = jerrors.New("session not exist")
 	errClientClosed      = jerrors.New("client closed")
 	errClientReadTimeout = jerrors.New("client read timeout")
+
+	clientConf *ClientConfig
 )
 
+const CONF_DUBBO_CLIENT_FILE_PATH = "CONF_DUBBO_CLIENT_FILE_PATH"
+
 func init() {
-	rand.Seed(time.Now().UnixNano())
+
+	// load clientconfig from *.yml
+	path := os.Getenv(CONF_DUBBO_CLIENT_FILE_PATH)
+	if path == "" {
+		log.Warn("CONF_CLIENT_FILE_PATH is null")
+		return
+	}
+
+	file, err := ioutil.ReadFile(path)
+	if err != nil {
+		log.Warn(jerrors.Trace(err))
+		return
+	}
+
+	conf := &ClientConfig{}
+	err = yaml.Unmarshal(file, conf)
+	if err != nil {
+		log.Warn(jerrors.Trace(err))
+		return
+	}
+
+	if err := conf.CheckValidity(); err != nil {
+		log.Warn("ClientConfig check failed: ", jerrors.Trace(err))
+		return
+	}
+
+	clientConf = conf
+
+}
+
+func SetClientConf(c ClientConfig) {
+	clientConf = &c
+}
+
+func GetClientConf() ClientConfig {
+	return *clientConf
 }
 
 type CallOptions struct {
@@ -61,6 +103,12 @@ func WithCallSerialID(s SerialID) CallOption {
 	}
 }
 
+func WithCallMeta_All(callMeta map[interface{}]interface{}) CallOption {
+	return func(o *CallOptions) {
+		o.Meta = callMeta
+	}
+}
+
 func WithCallMeta(k, v interface{}) CallOption {
 	return func(o *CallOptions) {
 		if o.Meta == nil {
@@ -89,22 +137,19 @@ type Client struct {
 	pendingResponses map[SequenceType]*PendingResponse
 }
 
-func NewClient(conf *ClientConfig) (*Client, error) {
-	if err := conf.CheckValidity(); err != nil {
-		return nil, jerrors.Trace(err)
-	}
+func NewClient() *Client {
 
 	c := &Client{
 		pendingResponses: make(map[SequenceType]*PendingResponse),
-		conf:             *conf,
+		conf:             *clientConf,
 	}
-	c.pool = newGettyRPCClientConnPool(c, conf.PoolSize, time.Duration(int(time.Second)*conf.PoolTTL))
+	c.pool = newGettyRPCClientConnPool(c, clientConf.PoolSize, time.Duration(int(time.Second)*clientConf.PoolTTL))
 
-	return c, nil
+	return c
 }
 
 // call one way
-func (c *Client) CallOneway(addr string, svcUrl registry.ServiceURL, method string, args interface{}, opts ...CallOption) error {
+func (c *Client) CallOneway(addr string, svcUrl config.URL, method string, args interface{}, opts ...CallOption) error {
 	var copts CallOptions
 
 	for _, o := range opts {
@@ -115,7 +160,7 @@ func (c *Client) CallOneway(addr string, svcUrl registry.ServiceURL, method stri
 }
 
 // if @reply is nil, the transport layer will get the response without notify the invoker.
-func (c *Client) Call(addr string, svcUrl registry.ServiceURL, method string, args, reply interface{}, opts ...CallOption) error {
+func (c *Client) Call(addr string, svcUrl config.URL, method string, args, reply interface{}, opts ...CallOption) error {
 	var copts CallOptions
 
 	for _, o := range opts {
@@ -130,7 +175,7 @@ func (c *Client) Call(addr string, svcUrl registry.ServiceURL, method string, ar
 	return jerrors.Trace(c.call(ct, addr, svcUrl, method, args, reply, nil, copts))
 }
 
-func (c *Client) AsyncCall(addr string, svcUrl registry.ServiceURL, method string, args interface{},
+func (c *Client) AsyncCall(addr string, svcUrl config.URL, method string, args interface{},
 	callback AsyncCallback, reply interface{}, opts ...CallOption) error {
 
 	var copts CallOptions
@@ -141,7 +186,7 @@ func (c *Client) AsyncCall(addr string, svcUrl registry.ServiceURL, method strin
 	return jerrors.Trace(c.call(CT_TwoWay, addr, svcUrl, method, args, reply, callback, copts))
 }
 
-func (c *Client) call(ct CallType, addr string, svcUrl registry.ServiceURL, method string,
+func (c *Client) call(ct CallType, addr string, svcUrl config.URL, method string,
 	args, reply interface{}, callback AsyncCallback, opts CallOptions) error {
 
 	if opts.RequestTimeout == 0 {
@@ -152,9 +197,9 @@ func (c *Client) call(ct CallType, addr string, svcUrl registry.ServiceURL, meth
 	}
 
 	p := &DubboPackage{}
-	p.Service.Path = strings.TrimPrefix(svcUrl.Path(), "/")
-	p.Service.Target = strings.TrimPrefix(svcUrl.Path(), "/")
-	p.Service.Version = svcUrl.Version()
+	p.Service.Path = strings.TrimPrefix(svcUrl.Path, "/")
+	p.Service.Target = svcUrl.GetParam(constant.INTERFACE_KEY, "")
+	p.Service.Version = svcUrl.GetParam(constant.VERSION_KEY, constant.DEFAULT_VERSION)
 	p.Service.Method = method
 	p.Service.Timeout = opts.RequestTimeout
 	if opts.SerialID == 0 {
@@ -210,7 +255,7 @@ func (c *Client) Close() {
 }
 
 func (c *Client) selectSession(addr string) (*gettyRPCClient, getty.Session, error) {
-	rpcClient, err := c.pool.getGettyRpcClient(public.CODECTYPE_DUBBO.String(), addr)
+	rpcClient, err := c.pool.getGettyRpcClient(DUBBO, addr)
 	if err != nil {
 		return nil, nil, jerrors.Trace(err)
 	}
diff --git a/dubbo/codec.go b/protocol/dubbo/codec.go
similarity index 95%
rename from dubbo/codec.go
rename to protocol/dubbo/codec.go
index d1a96940a4606377f6e42a122bbb8f91c789c556..cc1a215098abfeb5e4455bff990972d06a5150df 100644
--- a/dubbo/codec.go
+++ b/protocol/dubbo/codec.go
@@ -42,7 +42,7 @@ type DubboPackage struct {
 }
 
 func (p DubboPackage) String() string {
-	return fmt.Sprintf("DubboPackage: Header-%v, Service-%v, Body-%v", p.Header, p.Service, p.Body)
+	return fmt.Sprintf("DubboPackage: Header-%v, Path-%v, Body-%v", p.Header, p.Service, p.Body)
 }
 
 func (p *DubboPackage) Marshal() (*bytes.Buffer, error) {
diff --git a/dubbo/config.go b/protocol/dubbo/config.go
similarity index 98%
rename from dubbo/config.go
rename to protocol/dubbo/config.go
index 8405d0522772dda6be6190810a2b0a31a6c88531..3d0d5116c49cea3b7035b1aeee7d79f66802dde7 100644
--- a/dubbo/config.go
+++ b/protocol/dubbo/config.go
@@ -5,7 +5,6 @@ import (
 )
 
 import (
-	"github.com/dubbo/dubbo-go/server"
 	jerrors "github.com/juju/errors"
 )
 
@@ -32,7 +31,6 @@ type (
 
 	// Config holds supported types by the multiconfig package
 	ServerConfig struct {
-		server.ServerConfig
 		// local address
 		//AppName     string   `default:"rpc-server" yaml:"app_name" json:"app_name,omitempty"`
 		//Host        string   `default:"127.0.0.1" yaml:"host" json:"host,omitempty"`
diff --git a/protocol/dubbo/dubbo_exporter.go b/protocol/dubbo/dubbo_exporter.go
new file mode 100644
index 0000000000000000000000000000000000000000..9a4ed023f05c221ea731e6816ef771cedb494e11
--- /dev/null
+++ b/protocol/dubbo/dubbo_exporter.go
@@ -0,0 +1,19 @@
+package dubbo
+
+import (
+	"sync"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type DubboExporter struct {
+	protocol.BaseExporter
+}
+
+func NewDubboExporter(key string, invoker protocol.Invoker, exporterMap *sync.Map) *DubboExporter {
+	return &DubboExporter{
+		BaseExporter: *protocol.NewBaseExporter(key, invoker, exporterMap),
+	}
+}
diff --git a/protocol/dubbo/dubbo_invoker.go b/protocol/dubbo/dubbo_invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..51b9ab095849307fc5ccc7edccca8f9ae663f492
--- /dev/null
+++ b/protocol/dubbo/dubbo_invoker.go
@@ -0,0 +1,87 @@
+package dubbo
+
+import (
+	"errors"
+	"strconv"
+	"sync"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/support"
+)
+
+var Err_No_Reply = errors.New("request need @reply")
+
+type DubboInvoker struct {
+	protocol.BaseInvoker
+	client      *Client
+	destroyLock sync.Mutex
+}
+
+func NewDubboInvoker(url config.URL, client *Client) *DubboInvoker {
+	return &DubboInvoker{
+		BaseInvoker: *protocol.NewBaseInvoker(url),
+		client:      client,
+	}
+}
+
+func (di *DubboInvoker) Invoke(invocation protocol.Invocation) protocol.Result {
+
+	var (
+		err    error
+		result protocol.RPCResult
+	)
+
+	inv := invocation.(*support.RPCInvocation)
+	url := di.GetUrl()
+	// async
+	async, err := strconv.ParseBool(inv.AttachmentsByKey(constant.ASYNC_KEY, "false"))
+	if err != nil {
+		log.Error("ParseBool - error: %v", err)
+		async = false
+	}
+	if async {
+		if callBack, ok := inv.CallBack().(func(response CallResponse)); ok {
+			result.Err = di.client.AsyncCall(url.Location, url, inv.MethodName(), inv.Arguments(), callBack, inv.Reply())
+		} else {
+			result.Err = di.client.CallOneway(url.Location, url, inv.MethodName(), inv.Arguments())
+		}
+		log.Debug("result.Err: %v, result.Rest: %v", result.Err, result.Rest)
+	} else {
+		if inv.Reply() == nil {
+			result.Err = Err_No_Reply
+		} else {
+
+			result.Err = di.client.Call(url.Location, url, inv.MethodName(), inv.Arguments(), inv.Reply())
+			result.Rest = inv.Reply() // reply should be set to result.Rest when sync
+		}
+		log.Debug("result.Err: %v, result.Rest: %v", result.Err, result.Rest)
+	}
+
+	return &result
+}
+
+func (di *DubboInvoker) Destroy() {
+	if di.IsDestroyed() {
+		return
+	}
+	di.destroyLock.Lock()
+	defer di.destroyLock.Unlock()
+
+	if di.IsDestroyed() {
+		return
+	}
+
+	di.BaseInvoker.Destroy()
+
+	if di.client != nil {
+		di.client.Close() // close client
+	}
+}
diff --git a/protocol/dubbo/dubbo_protocol.go b/protocol/dubbo/dubbo_protocol.go
new file mode 100644
index 0000000000000000000000000000000000000000..3ab1bcd21c7418470cafc599839b76e85e80b673
--- /dev/null
+++ b/protocol/dubbo/dubbo_protocol.go
@@ -0,0 +1,79 @@
+package dubbo
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+const DUBBO = "dubbo"
+
+func init() {
+	extension.SetProtocol(DUBBO, GetProtocol)
+}
+
+var dubboProtocol *DubboProtocol
+
+type DubboProtocol struct {
+	protocol.BaseProtocol
+	serverMap map[string]*Server
+}
+
+func NewDubboProtocol() *DubboProtocol {
+	return &DubboProtocol{
+		BaseProtocol: protocol.NewBaseProtocol(),
+		serverMap:    make(map[string]*Server),
+	}
+}
+
+func (dp *DubboProtocol) Export(invoker protocol.Invoker) protocol.Exporter {
+	url := invoker.GetUrl()
+	serviceKey := url.Key()
+	exporter := NewDubboExporter(serviceKey, invoker, dp.ExporterMap())
+	dp.SetExporterMap(serviceKey, exporter)
+	log.Info("Export service: %s", url.String())
+
+	// start server
+	dp.openServer(url)
+	return exporter
+}
+
+func (dp *DubboProtocol) Refer(url config.URL) protocol.Invoker {
+	invoker := NewDubboInvoker(url, NewClient())
+	dp.SetInvokers(invoker)
+	log.Info("Refer service: %s", url.String())
+	return invoker
+}
+
+func (dp *DubboProtocol) Destroy() {
+	log.Info("DubboProtocol destroy.")
+
+	dp.BaseProtocol.Destroy()
+
+	// stop server
+	for key, server := range dp.serverMap {
+		delete(dp.serverMap, key)
+		server.Stop()
+	}
+}
+
+func (dp *DubboProtocol) openServer(url config.URL) {
+	exporter, ok := dp.ExporterMap().Load(url.Key())
+	if !ok {
+		panic("[DubboProtocol]" + url.Key() + "is not existing")
+	}
+	srv := NewServer(exporter.(protocol.Exporter))
+	dp.serverMap[url.Location] = srv
+	srv.Start(url)
+}
+
+func GetProtocol() protocol.Protocol {
+	if dubboProtocol != nil {
+		return dubboProtocol
+	}
+	return NewDubboProtocol()
+}
diff --git a/protocol/dubbo/dubbo_protocol_test.go b/protocol/dubbo/dubbo_protocol_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..ecd34d360cddfb7700b02fb1ae10f257497551c2
--- /dev/null
+++ b/protocol/dubbo/dubbo_protocol_test.go
@@ -0,0 +1,70 @@
+package dubbo
+
+import (
+	"context"
+	"testing"
+)
+
+import (
+	"github.com/stretchr/testify/assert"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+func TestDubboProtocol_Export(t *testing.T) {
+	// Export
+	proto := GetProtocol()
+	url, err := config.NewURL(context.Background(), "dubbo://192.168.56.1:20000/com.ikurento.user.UserProvider?anyhost=true&"+
+		"application=BDTService&category=providers&default.timeout=10000&dubbo=dubbo-provider-golang-1.0.0&"+
+		"environment=dev&interface=com.ikurento.user.UserProvider&ip=192.168.56.1&methods=GetUser%2C&"+
+		"module=dubbogo+user-info+server&org=ikurento.com&owner=ZX&pid=1447&revision=0.0.1&"+
+		"side=provider&timeout=3000&timestamp=1556509797245")
+	assert.NoError(t, err)
+	srvConf = &ServerConfig{}
+	exporter := proto.Export(protocol.NewBaseInvoker(url))
+
+	// make sure url
+	eq := exporter.GetInvoker().GetUrl().URLEqual(url)
+	assert.True(t, eq)
+
+	// make sure exporterMap after 'Unexport'
+	_, ok := proto.(*DubboProtocol).ExporterMap().Load(url.Key())
+	assert.True(t, ok)
+	exporter.Unexport()
+	_, ok = proto.(*DubboProtocol).ExporterMap().Load(url.Key())
+	assert.False(t, ok)
+
+	// make sure serverMap after 'Destroy'
+	_, ok = proto.(*DubboProtocol).serverMap[url.Location]
+	assert.True(t, ok)
+	proto.Destroy()
+	_, ok = proto.(*DubboProtocol).serverMap[url.Location]
+	assert.False(t, ok)
+}
+
+func TestDubboProtocol_Refer(t *testing.T) {
+	// Refer
+	proto := GetProtocol()
+	url, err := config.NewURL(context.Background(), "dubbo://192.168.56.1:20000/com.ikurento.user.UserProvider?anyhost=true&"+
+		"application=BDTService&category=providers&default.timeout=10000&dubbo=dubbo-provider-golang-1.0.0&"+
+		"environment=dev&interface=com.ikurento.user.UserProvider&ip=192.168.56.1&methods=GetUser%2C&"+
+		"module=dubbogo+user-info+server&org=ikurento.com&owner=ZX&pid=1447&revision=0.0.1&"+
+		"side=provider&timeout=3000&timestamp=1556509797245")
+	assert.NoError(t, err)
+	clientConf = &ClientConfig{}
+	invoker := proto.Refer(url)
+
+	// make sure url
+	eq := invoker.GetUrl().URLEqual(url)
+	assert.True(t, eq)
+
+	// make sure invokers after 'Destroy'
+	invokersLen := len(proto.(*DubboProtocol).Invokers())
+	assert.Equal(t, 1, invokersLen)
+	proto.Destroy()
+	invokersLen = len(proto.(*DubboProtocol).Invokers())
+	assert.Equal(t, 0, invokersLen)
+}
diff --git a/dubbo/listener.go b/protocol/dubbo/listener.go
similarity index 79%
rename from dubbo/listener.go
rename to protocol/dubbo/listener.go
index d97d88b00015111bfaf9778a1d31d00bc8dfe3cc..5a78ee180c01f77821b0161f04d772b98e3f3d0f 100644
--- a/dubbo/listener.go
+++ b/protocol/dubbo/listener.go
@@ -14,6 +14,12 @@ import (
 	jerrors "github.com/juju/errors"
 )
 
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/support"
+)
+
 // todo: WritePkg_Timeout will entry *.yml
 const WritePkg_Timeout = 5 * time.Second
 
@@ -106,14 +112,16 @@ func (h *RpcClientHandler) OnCron(session getty.Session) {
 ////////////////////////////////////////////
 
 type RpcServerHandler struct {
+	exporter       protocol.Exporter
 	maxSessionNum  int
 	sessionTimeout time.Duration
 	sessionMap     map[getty.Session]*rpcSession
 	rwlock         sync.RWMutex
 }
 
-func NewRpcServerHandler(maxSessionNum int, sessionTimeout time.Duration) *RpcServerHandler {
+func NewRpcServerHandler(exporter protocol.Exporter, maxSessionNum int, sessionTimeout time.Duration) *RpcServerHandler {
 	return &RpcServerHandler{
+		exporter:       exporter,
 		maxSessionNum:  maxSessionNum,
 		sessionTimeout: sessionTimeout,
 		sessionMap:     make(map[getty.Session]*rpcSession),
@@ -166,6 +174,18 @@ func (h *RpcServerHandler) OnMessage(session getty.Session, pkg interface{}) {
 	}
 	p.Header.ResponseStatus = hessian.Response_OK
 
+	invoker := h.exporter.GetInvoker()
+	if invoker != nil {
+		url := invoker.GetUrl()
+
+		result := invoker.Invoke(support.NewRPCInvocationForProvider(url))
+		if err := result.Error(); err != nil {
+			p.Header.ResponseStatus = hessian.Response_SERVER_ERROR
+			p.Body = err
+			return
+		}
+	}
+
 	// heartbeat
 	if p.Header.Type&hessian.Heartbeat != 0x00 {
 		log.Debug("get rpc heartbeat request{header: %#v, service: %#v, body: %#v}", p.Header, p.Service, p.Body)
@@ -173,7 +193,7 @@ func (h *RpcServerHandler) OnMessage(session getty.Session, pkg interface{}) {
 		return
 	}
 
-	// twoway
+	// not twoway
 	if p.Header.Type&hessian.Request_TwoWay == 0x00 {
 		h.reply(session, p, hessian.Response)
 		h.callService(p, nil)
@@ -227,16 +247,29 @@ func (h *RpcServerHandler) callService(req *DubboPackage, ctx context.Context) {
 		}
 	}()
 
-	svc := req.Body.(map[string]interface{})["service"].(*service)
-	method := svc.method[req.Service.Method]
+	svcIf := req.Body.(map[string]interface{})["service"]
+	if svcIf == nil {
+		log.Error("service not found!")
+		req.Header.ResponseStatus = hessian.Response_SERVICE_NOT_FOUND
+		req.Body = nil
+		return
+	}
+	svc := svcIf.(*config.Service)
+	method := svc.Method()[req.Service.Method]
+	if method == nil {
+		log.Error("method not found!")
+		req.Header.ResponseStatus = hessian.Response_SERVICE_NOT_FOUND
+		req.Body = nil
+		return
+	}
 
 	// prepare argv
 	var argv reflect.Value
 	argIsValue := false // if true, need to indirect before calling.
-	if method.ArgType.Kind() == reflect.Ptr {
-		argv = reflect.New(method.ArgType.Elem())
+	if method.ArgType().Kind() == reflect.Ptr {
+		argv = reflect.New(method.ArgType().Elem())
 	} else {
-		argv = reflect.New(method.ArgType)
+		argv = reflect.New(method.ArgType())
 		argIsValue = true
 	}
 	argvTmp := argv.Interface()
@@ -246,15 +279,15 @@ func (h *RpcServerHandler) callService(req *DubboPackage, ctx context.Context) {
 	}
 
 	// prepare replyv
-	replyv := reflect.New(method.ReplyType.Elem())
+	replyv := reflect.New(method.ReplyType().Elem())
 	var returnValues []reflect.Value
-	if method.CtxType == nil {
-		returnValues = method.method.Func.Call([]reflect.Value{svc.rcvr, reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
+	if method.CtxType() == nil {
+		returnValues = method.Method().Func.Call([]reflect.Value{svc.Rcvr(), reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
 	} else {
 		if contextv := reflect.ValueOf(ctx); contextv.IsValid() {
-			returnValues = method.method.Func.Call([]reflect.Value{svc.rcvr, contextv, reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
+			returnValues = method.Method().Func.Call([]reflect.Value{svc.Rcvr(), contextv, reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
 		} else {
-			returnValues = method.method.Func.Call([]reflect.Value{svc.rcvr, reflect.Zero(method.CtxType), reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
+			returnValues = method.Method().Func.Call([]reflect.Value{svc.Rcvr(), reflect.Zero(method.CtxType()), reflect.ValueOf(argvTmp), reflect.ValueOf(replyv.Interface())})
 		}
 	}
 
diff --git a/dubbo/pool.go b/protocol/dubbo/pool.go
similarity index 95%
rename from dubbo/pool.go
rename to protocol/dubbo/pool.go
index 2cc9cc7ac99142b2ad32beea6cc39722d0dd9a39..1f9828d7b7f28bbe0758a3c0a686fe992fa67133 100644
--- a/dubbo/pool.go
+++ b/protocol/dubbo/pool.go
@@ -250,13 +250,8 @@ func (p *gettyRPCClientPool) close() {
 }
 
 func (p *gettyRPCClientPool) getGettyRpcClient(protocol, addr string) (*gettyRPCClient, error) {
-	var builder strings.Builder
 
-	builder.WriteString(addr)
-	builder.WriteString("@")
-	builder.WriteString(protocol)
-
-	key := builder.String()
+	key := GenerateEndpointAddr(protocol, addr)
 
 	p.Lock()
 	defer p.Unlock()
@@ -293,13 +288,7 @@ func (p *gettyRPCClientPool) release(conn *gettyRPCClient, err error) {
 		return
 	}
 
-	var builder strings.Builder
-
-	builder.WriteString(conn.addr)
-	builder.WriteString("@")
-	builder.WriteString(conn.protocol)
-
-	key := builder.String()
+	key := GenerateEndpointAddr(conn.protocol, conn.addr)
 
 	p.Lock()
 	defer p.Unlock()
@@ -320,13 +309,7 @@ func (p *gettyRPCClientPool) remove(conn *gettyRPCClient) {
 		return
 	}
 
-	var builder strings.Builder
-
-	builder.WriteString(conn.addr)
-	builder.WriteString("@")
-	builder.WriteString(conn.protocol)
-
-	key := builder.String()
+	key := GenerateEndpointAddr(conn.protocol, conn.addr)
 
 	p.Lock()
 	defer p.Unlock()
@@ -344,3 +327,13 @@ func (p *gettyRPCClientPool) remove(conn *gettyRPCClient) {
 		}
 	}
 }
+
+func GenerateEndpointAddr(protocol, addr string) string {
+	var builder strings.Builder
+
+	builder.WriteString(protocol)
+	builder.WriteString("://")
+	builder.WriteString(addr)
+
+	return builder.String()
+}
diff --git a/dubbo/readwriter.go b/protocol/dubbo/readwriter.go
similarity index 92%
rename from dubbo/readwriter.go
rename to protocol/dubbo/readwriter.go
index e14cf201009a98992d57ed1e714a7cb3252bd862..8338043ff57a95bdbbcd93e26777c1859cc10eb8 100644
--- a/dubbo/readwriter.go
+++ b/protocol/dubbo/readwriter.go
@@ -10,6 +10,9 @@ import (
 	log "github.com/AlexStocks/log4go"
 	jerrors "github.com/juju/errors"
 )
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
 
 ////////////////////////////////////////////
 // RpcClientPackageHandler
@@ -59,15 +62,10 @@ func (p *RpcClientPackageHandler) Write(ss getty.Session, pkg interface{}) error
 ////////////////////////////////////////////
 
 type RpcServerPackageHandler struct {
-	server *Server
-	srvMap serviceMap
 }
 
-func NewRpcServerPackageHandler(server *Server, srvMap serviceMap) *RpcServerPackageHandler {
-	return &RpcServerPackageHandler{
-		server: server,
-		srvMap: srvMap,
-	}
+func NewRpcServerPackageHandler() *RpcServerPackageHandler {
+	return &RpcServerPackageHandler{}
 }
 
 func (p *RpcServerPackageHandler) Read(ss getty.Session, data []byte) (interface{}, int, error) {
@@ -111,7 +109,7 @@ func (p *RpcServerPackageHandler) Read(ss getty.Session, data []byte) (interface
 			"dubboVersion": dubboVersion,
 			"argsTypes":    argsTypes,
 			"args":         args,
-			"service":      p.srvMap[pkg.Service.Target],
+			"service":      config.ServiceMap.GetService(DUBBO, pkg.Service.Target),
 			"attachments":  attachments,
 		}
 	}
diff --git a/protocol/dubbo/server.go b/protocol/dubbo/server.go
new file mode 100644
index 0000000000000000000000000000000000000000..7f5bd3c82f0cf2f5aba7083ee88fc767a4f62ee8
--- /dev/null
+++ b/protocol/dubbo/server.go
@@ -0,0 +1,135 @@
+package dubbo
+
+import (
+	"fmt"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"gopkg.in/yaml.v2"
+	"io/ioutil"
+	"net"
+	"os"
+)
+
+import (
+	"github.com/AlexStocks/getty"
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
+
+var srvConf *ServerConfig
+
+const CONF_DUBBO_SERVER_FILE_PATH = "CONF_DUBBO_SERVER_FILE_PATH"
+
+func init() {
+	// load serverconfig from *.yml
+	path := os.Getenv(CONF_DUBBO_SERVER_FILE_PATH)
+	if path == "" {
+		log.Warn("CONF_SERVER_FILE_PATH is null")
+		return
+	}
+
+	file, err := ioutil.ReadFile(path)
+	if err != nil {
+		log.Warn(jerrors.Trace(err))
+		return
+	}
+
+	conf := &ServerConfig{}
+	err = yaml.Unmarshal(file, conf)
+	if err != nil {
+		log.Warn(jerrors.Trace(err))
+		return
+	}
+
+	if err := conf.CheckValidity(); err != nil {
+		log.Warn("ServerConfig check failed: ", err)
+		return
+	}
+
+	srvConf = conf
+}
+
+func SetServerConfig(s ServerConfig) {
+	srvConf = &s
+}
+
+func GetServerConfig() ServerConfig {
+	return *srvConf
+}
+
+type Server struct {
+	conf      ServerConfig
+	tcpServer getty.Server
+	exporter  protocol.Exporter
+}
+
+func NewServer(exporter protocol.Exporter) *Server {
+
+	s := &Server{
+		exporter: exporter,
+		conf:     *srvConf,
+	}
+
+	return s
+}
+
+func (s *Server) newSession(session getty.Session) error {
+	var (
+		ok      bool
+		tcpConn *net.TCPConn
+	)
+	conf := s.conf
+
+	if conf.GettySessionParam.CompressEncoding {
+		session.SetCompressType(getty.CompressZip)
+	}
+
+	if tcpConn, ok = session.Conn().(*net.TCPConn); !ok {
+		panic(fmt.Sprintf("%s, session.conn{%#v} is not tcp connection\n", session.Stat(), session.Conn()))
+	}
+
+	tcpConn.SetNoDelay(conf.GettySessionParam.TcpNoDelay)
+	tcpConn.SetKeepAlive(conf.GettySessionParam.TcpKeepAlive)
+	if conf.GettySessionParam.TcpKeepAlive {
+		tcpConn.SetKeepAlivePeriod(conf.GettySessionParam.keepAlivePeriod)
+	}
+	tcpConn.SetReadBuffer(conf.GettySessionParam.TcpRBufSize)
+	tcpConn.SetWriteBuffer(conf.GettySessionParam.TcpWBufSize)
+
+	session.SetName(conf.GettySessionParam.SessionName)
+	session.SetMaxMsgLen(conf.GettySessionParam.MaxMsgLen)
+	session.SetPkgHandler(NewRpcServerPackageHandler())
+	session.SetEventListener(NewRpcServerHandler(s.exporter, conf.SessionNumber, conf.sessionTimeout))
+	session.SetRQLen(conf.GettySessionParam.PkgRQSize)
+	session.SetWQLen(conf.GettySessionParam.PkgWQSize)
+	session.SetReadTimeout(conf.GettySessionParam.tcpReadTimeout)
+	session.SetWriteTimeout(conf.GettySessionParam.tcpWriteTimeout)
+	session.SetCronPeriod((int)(conf.sessionTimeout.Nanoseconds() / 1e6))
+	session.SetWaitTime(conf.GettySessionParam.waitTimeout)
+	log.Debug("app accepts new session:%s\n", session.Stat())
+
+	return nil
+}
+
+func (s *Server) Start(url config.URL) {
+	var (
+		addr      string
+		tcpServer getty.Server
+	)
+
+	addr = url.Location
+	tcpServer = getty.NewTCPServer(
+		getty.WithLocalAddress(addr),
+	)
+	tcpServer.RunEventLoop(s.newSession)
+	log.Debug("s bind addr{%s} ok!", addr)
+	s.tcpServer = tcpServer
+
+}
+
+func (s *Server) Stop() {
+	s.tcpServer.Close()
+}
diff --git a/protocol/invocation.go b/protocol/invocation.go
new file mode 100644
index 0000000000000000000000000000000000000000..b5725be5c03e6cf603270ac2e605d8c4a242de51
--- /dev/null
+++ b/protocol/invocation.go
@@ -0,0 +1,15 @@
+package protocol
+
+import (
+	"reflect"
+)
+
+type Invocation interface {
+	MethodName() string
+	ParameterTypes() []reflect.Type
+	Arguments() []interface{}
+	Reply() interface{}
+	Attachments() map[string]string
+	AttachmentsByKey(string, string) string
+	Invoker() Invoker
+}
diff --git a/protocol/invoker.go b/protocol/invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..5611bb31f759c669617157a472613beffe4571ac
--- /dev/null
+++ b/protocol/invoker.go
@@ -0,0 +1,56 @@
+package protocol
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
+
+// Extension - Invoker
+type Invoker interface {
+	common.Node
+	Invoke(Invocation) Result
+}
+
+/////////////////////////////
+// base invoker
+/////////////////////////////
+
+type BaseInvoker struct {
+	url       config.URL
+	available bool
+	destroyed bool
+}
+
+func NewBaseInvoker(url config.URL) *BaseInvoker {
+	return &BaseInvoker{
+		url:       url,
+		available: true,
+		destroyed: false,
+	}
+}
+
+func (bi *BaseInvoker) GetUrl() config.URL {
+	return bi.url
+}
+
+func (bi *BaseInvoker) IsAvailable() bool {
+	return bi.available
+}
+
+func (bi *BaseInvoker) IsDestroyed() bool {
+	return bi.destroyed
+}
+
+func (bi *BaseInvoker) Invoke(invocation Invocation) Result {
+	return &RPCResult{}
+}
+
+func (bi *BaseInvoker) Destroy() {
+	log.Info("Destroy invoker: %s", bi.GetUrl().String())
+	bi.destroyed = true
+	bi.available = false
+}
diff --git a/jsonrpc/http.go b/protocol/jsonrpc/http.go
similarity index 80%
rename from jsonrpc/http.go
rename to protocol/jsonrpc/http.go
index 55fee69d3d3bd8a0d2360fae40f9c561d8e6997f..7e19e3742aa0c74bc14e611b3bd439dad305188d 100644
--- a/jsonrpc/http.go
+++ b/protocol/jsonrpc/http.go
@@ -20,9 +20,8 @@ import (
 )
 
 import (
-	"github.com/dubbo/dubbo-go/client"
-	"github.com/dubbo/dubbo-go/public"
-	"github.com/dubbo/dubbo-go/registry"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
 )
 
 //////////////////////////////////////////////
@@ -38,11 +37,6 @@ type Request struct {
 	method      string
 	args        interface{}
 	contentType string
-	conf        registry.ServiceConfig
-}
-
-func (r *Request) ServiceConfig() registry.ServiceConfig {
-	return r.conf
 }
 
 //////////////////////////////////////////////
@@ -83,36 +77,31 @@ func NewHTTPClient(opt *HTTPOptions) *HTTPClient {
 	}
 }
 
-func (c *HTTPClient) NewRequest(conf registry.ServiceConfig, method string, args interface{}) (client.Request, error) {
+func (c *HTTPClient) NewRequest(service config.URL, method string, args interface{}) *Request {
 
 	return &Request{
 		ID:       atomic.AddInt64(&c.ID, 1),
-		group:    conf.Group(),
-		protocol: conf.Protocol(),
-		version:  conf.Version(),
-		service:  conf.Service(),
+		group:    service.GetParam(constant.GROUP_KEY, ""),
+		protocol: service.Protocol,
+		version:  service.GetParam(constant.VERSION_KEY, constant.DEFAULT_VERSION),
+		service:  service.Path,
 		method:   method,
 		args:     args,
-		conf:     conf,
-	}, nil
+	}
 }
 
-func (c *HTTPClient) Call(ctx context.Context, service registry.ServiceURL, request client.Request, rsp interface{}) error {
+func (c *HTTPClient) Call(ctx context.Context, service config.URL, req *Request, rsp interface{}) error {
 	// header
-	req := request.(*Request)
 	httpHeader := http.Header{}
 	httpHeader.Set("Content-Type", "application/json")
 	httpHeader.Set("Accept", "application/json")
 
 	reqTimeout := c.options.HTTPTimeout
-	if service.Timeout() != 0 && service.Timeout() < reqTimeout {
-		reqTimeout = time.Duration(service.Timeout())
-	}
 	if reqTimeout <= 0 {
 		reqTimeout = 1e8
 	}
 	httpHeader.Set("Timeout", reqTimeout.String())
-	if md, ok := ctx.Value(public.DUBBOGO_CTX_KEY).(map[string]string); ok {
+	if md, ok := ctx.Value(constant.DUBBOGO_CTX_KEY).(map[string]string); ok {
 		for k := range md {
 			httpHeader.Set(k, md[k])
 		}
@@ -130,7 +119,7 @@ func (c *HTTPClient) Call(ctx context.Context, service registry.ServiceURL, requ
 		return jerrors.Trace(err)
 	}
 
-	rspBody, err := c.Do(service.Location(), service.Query().Get("interface"), httpHeader, reqBody)
+	rspBody, err := c.Do(service.Location, service.Params.Get("interface"), httpHeader, reqBody)
 	if err != nil {
 		return jerrors.Trace(err)
 	}
diff --git a/jsonrpc/json.go b/protocol/jsonrpc/json.go
similarity index 100%
rename from jsonrpc/json.go
rename to protocol/jsonrpc/json.go
diff --git a/protocol/jsonrpc/jsonrpc_exporter.go b/protocol/jsonrpc/jsonrpc_exporter.go
new file mode 100644
index 0000000000000000000000000000000000000000..3401d3fcb8774f4cc60c8b9148369a5b756afd32
--- /dev/null
+++ b/protocol/jsonrpc/jsonrpc_exporter.go
@@ -0,0 +1,19 @@
+package jsonrpc
+
+import (
+	"sync"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type JsonrpcExporter struct {
+	protocol.BaseExporter
+}
+
+func NewJsonrpcExporter(key string, invoker protocol.Invoker, exporterMap *sync.Map) *JsonrpcExporter {
+	return &JsonrpcExporter{
+		BaseExporter: *protocol.NewBaseExporter(key, invoker, exporterMap),
+	}
+}
diff --git a/protocol/jsonrpc/jsonrpc_invoker.go b/protocol/jsonrpc/jsonrpc_invoker.go
new file mode 100644
index 0000000000000000000000000000000000000000..8f2e197b2e2b09705b3aac1866346c5bc15d43bb
--- /dev/null
+++ b/protocol/jsonrpc/jsonrpc_invoker.go
@@ -0,0 +1,54 @@
+package jsonrpc
+
+import (
+	"context"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/support"
+)
+
+type JsonrpcInvoker struct {
+	protocol.BaseInvoker
+	client *HTTPClient
+}
+
+func NewJsonrpcInvoker(url config.URL, client *HTTPClient) *JsonrpcInvoker {
+	return &JsonrpcInvoker{
+		BaseInvoker: *protocol.NewBaseInvoker(url),
+		client:      client,
+	}
+}
+
+func (ji *JsonrpcInvoker) Invoke(invocation protocol.Invocation) protocol.Result {
+
+	var (
+		result protocol.RPCResult
+	)
+
+	inv := invocation.(*support.RPCInvocation)
+	url := ji.GetUrl()
+	req := ji.client.NewRequest(url, inv.MethodName(), inv.Arguments())
+	ctx := context.WithValue(context.Background(), constant.DUBBOGO_CTX_KEY, map[string]string{
+		"X-Proxy-Id": "dubbogo",
+		"X-Services": url.Path,
+		"X-Method":   inv.MethodName(),
+	})
+	if err := ji.client.Call(ctx, url, req, inv.Reply()); err != nil {
+		log.Error("client.Call() return error:%+v", jerrors.ErrorStack(err))
+		result.Err = err
+	} else {
+		log.Debug("result: %v", inv.Reply())
+		result.Rest = inv.Reply()
+	}
+
+	return &result
+}
diff --git a/protocol/jsonrpc/jsonrpc_protocol.go b/protocol/jsonrpc/jsonrpc_protocol.go
new file mode 100644
index 0000000000000000000000000000000000000000..98276e8c568394ec54ff588bc958b45471851eaf
--- /dev/null
+++ b/protocol/jsonrpc/jsonrpc_protocol.go
@@ -0,0 +1,84 @@
+package jsonrpc
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+const JSONRPC = "jsonrpc"
+
+func init() {
+	extension.SetProtocol(JSONRPC, GetProtocol)
+}
+
+var jsonrpcProtocol *JsonrpcProtocol
+
+type JsonrpcProtocol struct {
+	protocol.BaseProtocol
+	serverMap map[string]*Server
+}
+
+func NewDubboProtocol() *JsonrpcProtocol {
+	return &JsonrpcProtocol{
+		BaseProtocol: protocol.NewBaseProtocol(),
+		serverMap:    make(map[string]*Server),
+	}
+}
+
+func (jp *JsonrpcProtocol) Export(invoker protocol.Invoker) protocol.Exporter {
+	url := invoker.GetUrl()
+	serviceKey := url.Key()
+	exporter := NewJsonrpcExporter(serviceKey, invoker, jp.ExporterMap())
+	jp.SetExporterMap(serviceKey, exporter)
+	log.Info("Export service: %s", url.String())
+
+	// start server
+	jp.openServer(url)
+
+	return exporter
+}
+
+func (jp *JsonrpcProtocol) Refer(url config.URL) protocol.Invoker {
+	invoker := NewJsonrpcInvoker(url, NewHTTPClient(&HTTPOptions{
+		HandshakeTimeout: support.GetConsumerConfig().ConnectTimeout,
+		HTTPTimeout:      support.GetConsumerConfig().RequestTimeout,
+	}))
+	jp.SetInvokers(invoker)
+	log.Info("Refer service: %s", url.String())
+	return invoker
+}
+
+func (jp *JsonrpcProtocol) Destroy() {
+	log.Info("jsonrpcProtocol destroy.")
+
+	jp.BaseProtocol.Destroy()
+
+	// stop server
+	for key, server := range jp.serverMap {
+		delete(jp.serverMap, key)
+		server.Stop()
+	}
+}
+
+func (jp *JsonrpcProtocol) openServer(url config.URL) {
+	exporter, ok := jp.ExporterMap().Load(url.Key())
+	if !ok {
+		panic("[JsonrpcProtocol]" + url.Key() + "is not existing")
+	}
+	srv := NewServer(exporter.(protocol.Exporter))
+	jp.serverMap[url.Location] = srv
+	srv.Start(url)
+}
+
+func GetProtocol() protocol.Protocol {
+	if jsonrpcProtocol != nil {
+		return jsonrpcProtocol
+	}
+	return NewDubboProtocol()
+}
diff --git a/protocol/jsonrpc/jsonrpc_protocol_test.go b/protocol/jsonrpc/jsonrpc_protocol_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..80fc9219efe78270e8628dc1f4b3a9abe80b0227
--- /dev/null
+++ b/protocol/jsonrpc/jsonrpc_protocol_test.go
@@ -0,0 +1,75 @@
+package jsonrpc
+
+import (
+	"context"
+	"testing"
+	"time"
+)
+
+import (
+	"github.com/stretchr/testify/assert"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/config/support"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+func TestJsonrpcProtocol_Export(t *testing.T) {
+	// Export
+	proto := GetProtocol()
+	url, err := config.NewURL(context.Background(), "dubbo://192.168.56.1:20000/com.ikurento.user.UserProvider?anyhost=true&"+
+		"application=BDTService&category=providers&default.timeout=10000&dubbo=dubbo-provider-golang-1.0.0&"+
+		"environment=dev&interface=com.ikurento.user.UserProvider&ip=192.168.56.1&methods=GetUser%2C&"+
+		"module=dubbogo+user-info+server&org=ikurento.com&owner=ZX&pid=1447&revision=0.0.1&"+
+		"side=provider&timeout=3000&timestamp=1556509797245")
+	assert.NoError(t, err)
+	exporter := proto.Export(protocol.NewBaseInvoker(url))
+
+	// make sure url
+	eq := exporter.GetInvoker().GetUrl().URLEqual(url)
+	assert.True(t, eq)
+
+	// make sure exporterMap after 'Unexport'
+	_, ok := proto.(*JsonrpcProtocol).ExporterMap().Load(url.Key())
+	assert.True(t, ok)
+	exporter.Unexport()
+	_, ok = proto.(*JsonrpcProtocol).ExporterMap().Load(url.Key())
+	assert.False(t, ok)
+
+	// make sure serverMap after 'Destroy'
+	_, ok = proto.(*JsonrpcProtocol).serverMap[url.Location]
+	assert.True(t, ok)
+	proto.Destroy()
+	_, ok = proto.(*JsonrpcProtocol).serverMap[url.Location]
+	assert.False(t, ok)
+}
+
+func TestJsonrpcProtocol_Refer(t *testing.T) {
+	// Refer
+	proto := GetProtocol()
+	url, err := config.NewURL(context.Background(), "dubbo://192.168.56.1:20000/com.ikurento.user.UserProvider?anyhost=true&"+
+		"application=BDTService&category=providers&default.timeout=10000&dubbo=dubbo-provider-golang-1.0.0&"+
+		"environment=dev&interface=com.ikurento.user.UserProvider&ip=192.168.56.1&methods=GetUser%2C&"+
+		"module=dubbogo+user-info+server&org=ikurento.com&owner=ZX&pid=1447&revision=0.0.1&"+
+		"side=provider&timeout=3000&timestamp=1556509797245")
+	assert.NoError(t, err)
+	con := support.ConsumerConfig{
+		ConnectTimeout: 5 * time.Second,
+		RequestTimeout: 5 * time.Second,
+	}
+	support.SetConsumerConfig(con)
+	invoker := proto.Refer(url)
+
+	// make sure url
+	eq := invoker.GetUrl().URLEqual(url)
+	assert.True(t, eq)
+
+	// make sure invokers after 'Destroy'
+	invokersLen := len(proto.(*JsonrpcProtocol).Invokers())
+	assert.Equal(t, 1, invokersLen)
+	proto.Destroy()
+	invokersLen = len(proto.(*JsonrpcProtocol).Invokers())
+	assert.Equal(t, 0, invokersLen)
+}
diff --git a/protocol/jsonrpc/server.go b/protocol/jsonrpc/server.go
new file mode 100644
index 0000000000000000000000000000000000000000..78e2944e39e8452ca3ea5595ce0d8abbf8ed2682
--- /dev/null
+++ b/protocol/jsonrpc/server.go
@@ -0,0 +1,349 @@
+package jsonrpc
+
+import (
+	"bufio"
+	"bytes"
+	"context"
+	"io"
+	"io/ioutil"
+	"net"
+	"net/http"
+	"reflect"
+	"runtime"
+	"runtime/debug"
+	"sync"
+	"time"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/support"
+)
+
+var (
+	// A value sent as a placeholder for the server's response value when the server
+	// receives an invalid request. It is never decoded by the client since the Response
+	// contains an error when it is used.
+	invalidRequest = struct{}{}
+)
+
+const (
+	DefaultMaxSleepTime      = 1 * time.Second // accept中间最大sleep interval
+	DefaultHTTPRspBufferSize = 1024
+	PathPrefix               = byte('/')
+)
+
+type Server struct {
+	exporter protocol.Exporter
+	done     chan struct{}
+	once     sync.Once
+
+	sync.RWMutex
+	wg      sync.WaitGroup
+	timeout time.Duration
+}
+
+func NewServer(exporter protocol.Exporter) *Server {
+	return &Server{
+		exporter: exporter,
+		done:     make(chan struct{}),
+	}
+}
+
+func (s *Server) handlePkg(conn net.Conn) {
+	defer func() {
+		if r := recover(); r != nil {
+			log.Warn("connection{local:%v, remote:%v} panic error:%#v, debug stack:%s",
+				conn.LocalAddr(), conn.RemoteAddr(), r, string(debug.Stack()))
+		}
+
+		conn.Close()
+	}()
+
+	setReadTimeout := func(conn net.Conn, timeout time.Duration) {
+		t := time.Time{}
+		if timeout > time.Duration(0) {
+			t = time.Now().Add(timeout)
+		}
+
+		conn.SetDeadline(t)
+	}
+
+	sendErrorResp := func(header http.Header, body []byte) error {
+		rsp := &http.Response{
+			Header:        header,
+			StatusCode:    500,
+			ContentLength: int64(len(body)),
+			Body:          ioutil.NopCloser(bytes.NewReader(body)),
+		}
+		rsp.Header.Del("Content-Type")
+		rsp.Header.Del("Content-Length")
+		rsp.Header.Del("Timeout")
+
+		rspBuf := bytes.NewBuffer(make([]byte, DefaultHTTPRspBufferSize))
+		rspBuf.Reset()
+		err := rsp.Write(rspBuf)
+		if err != nil {
+			return jerrors.Trace(err)
+		}
+		_, err = rspBuf.WriteTo(conn)
+		return jerrors.Trace(err)
+	}
+
+	for {
+		bufReader := bufio.NewReader(conn)
+		r, err := http.ReadRequest(bufReader)
+		if err != nil {
+			return
+		}
+
+		reqBody, err := ioutil.ReadAll(r.Body)
+		if err != nil {
+			return
+		}
+		r.Body.Close()
+
+		reqHeader := make(map[string]string)
+		for k := range r.Header {
+			reqHeader[k] = r.Header.Get(k)
+		}
+		reqHeader["Path"] = r.URL.Path[1:] // to get service name
+		if r.URL.Path[0] != PathPrefix {
+			reqHeader["Path"] = r.URL.Path
+		}
+		reqHeader["HttpMethod"] = r.Method
+
+		httpTimeout := s.timeout
+		contentType := reqHeader["Content-Type"]
+		if contentType != "application/json" && contentType != "application/json-rpc" {
+			setReadTimeout(conn, httpTimeout)
+			r.Header.Set("Content-Type", "text/plain")
+			if errRsp := sendErrorResp(r.Header, []byte(jerrors.ErrorStack(err))); errRsp != nil {
+				log.Warn("sendErrorResp(header:%#v, error:%s) = error:%s",
+					r.Header, jerrors.ErrorStack(err), errRsp)
+			}
+			return
+		}
+
+		// exporter invoke
+		invoker := s.exporter.GetInvoker()
+		if invoker != nil {
+			url := invoker.GetUrl()
+
+			result := invoker.Invoke(support.NewRPCInvocationForProvider(url))
+			if err := result.Error(); err != nil {
+				if errRsp := sendErrorResp(r.Header, []byte(err.Error())); errRsp != nil {
+					log.Warn("Exporter: sendErrorResp(header:%#v, error:%v) = error:%s",
+						r.Header, err, errRsp)
+				}
+				return
+			}
+		}
+
+		ctx := context.Background()
+		if len(reqHeader["Timeout"]) > 0 {
+			timeout, err := time.ParseDuration(reqHeader["Timeout"])
+			if err == nil {
+				httpTimeout = timeout
+				ctx, _ = context.WithTimeout(ctx, httpTimeout)
+			}
+			delete(reqHeader, "Timeout")
+		}
+		setReadTimeout(conn, httpTimeout)
+
+		if err := serveRequest(ctx, reqHeader, reqBody, conn); err != nil {
+			if errRsp := sendErrorResp(r.Header, []byte(jerrors.ErrorStack(err))); errRsp != nil {
+				log.Warn("sendErrorResp(header:%#v, error:%s) = error:%s",
+					r.Header, jerrors.ErrorStack(err), errRsp)
+			}
+
+			log.Info("Unexpected error serving request, closing socket: %v", err)
+			return
+		}
+	}
+}
+
+func accept(listener net.Listener, fn func(net.Conn)) error {
+	var (
+		err      error
+		c        net.Conn
+		ok       bool
+		ne       net.Error
+		tmpDelay time.Duration
+	)
+
+	for {
+		c, err = listener.Accept()
+		if err != nil {
+			if ne, ok = err.(net.Error); ok && ne.Temporary() {
+				if tmpDelay != 0 {
+					tmpDelay <<= 1
+				} else {
+					tmpDelay = 5 * time.Millisecond
+				}
+				if tmpDelay > DefaultMaxSleepTime {
+					tmpDelay = DefaultMaxSleepTime
+				}
+				log.Info("http: Accept error: %v; retrying in %v\n", err, tmpDelay)
+				time.Sleep(tmpDelay)
+				continue
+			}
+			return jerrors.Trace(err)
+		}
+
+		go func() {
+			defer func() {
+				if r := recover(); r != nil {
+					const size = 64 << 10
+					buf := make([]byte, size)
+					buf = buf[:runtime.Stack(buf, false)]
+					log.Error("http: panic serving %v: %v\n%s", c.RemoteAddr(), r, buf)
+					c.Close()
+				}
+			}()
+
+			fn(c)
+		}()
+	}
+}
+
+func (s *Server) Start(url config.URL) {
+	listener, err := net.Listen("tcp", url.Location)
+	if err != nil {
+		log.Error("jsonrpc server [%s] start failed: %v", url.Path, err)
+		return
+	}
+	log.Info("rpc server start to listen on %s", listener.Addr())
+
+	s.wg.Add(1)
+	go func() {
+		accept(listener, func(conn net.Conn) { s.handlePkg(conn) })
+		s.wg.Done()
+	}()
+
+	s.wg.Add(1)
+	go func() { // Server done goroutine
+		var err error
+		<-s.done               // step1: block to wait for done channel(wait Server.Stop step2)
+		err = listener.Close() // step2: and then close listener
+		if err != nil {
+			log.Warn("listener{addr:%s}.Close() = error{%#v}", listener.Addr(), err)
+		}
+		s.wg.Done()
+	}()
+}
+
+func (s *Server) Stop() {
+	s.once.Do(func() {
+		close(s.done)
+		s.wg.Wait()
+	})
+}
+
+func serveRequest(ctx context.Context,
+	header map[string]string, body []byte, conn net.Conn) error {
+
+	// read request header
+	codec := newServerCodec()
+	err := codec.ReadHeader(header, body)
+	if err != nil {
+		if err == io.EOF || err == io.ErrUnexpectedEOF {
+			return jerrors.Trace(err)
+		}
+
+		return jerrors.New("server cannot decode request: " + err.Error())
+	}
+	serviceName := header["Path"]
+	methodName := codec.req.Method
+	if len(serviceName) == 0 || len(methodName) == 0 {
+		codec.ReadBody(nil)
+		return jerrors.New("service/method request ill-formed: " + serviceName + "/" + methodName)
+	}
+
+	// get method
+	svc := config.ServiceMap.GetService(JSONRPC, serviceName)
+	if svc == nil {
+		codec.ReadBody(nil)
+		return jerrors.New("cannot find svc " + serviceName)
+	}
+	mtype := svc.Method()[methodName]
+	if mtype == nil {
+		codec.ReadBody(nil)
+		return jerrors.New("cannot find method " + methodName + " of svc " + serviceName)
+	}
+
+	// get args
+	var argv reflect.Value
+	argIsValue := false
+	if mtype.ArgType().Kind() == reflect.Ptr {
+		argv = reflect.New(mtype.ArgType().Elem())
+	} else {
+		argv = reflect.New(mtype.ArgType())
+		argIsValue = true
+	}
+	// argv guaranteed to be a pointer now.
+	if err = codec.ReadBody(argv.Interface()); err != nil {
+		return jerrors.Trace(err)
+	}
+	if argIsValue {
+		argv = argv.Elem()
+	}
+
+	replyv := reflect.New(mtype.ReplyType().Elem())
+
+	//  call service.method(args)
+	var errMsg string
+	returnValues := mtype.Method().Func.Call([]reflect.Value{
+		svc.Rcvr(),
+		mtype.SuiteContext(ctx),
+		reflect.ValueOf(argv.Interface()),
+		reflect.ValueOf(replyv.Interface()),
+	})
+	// The return value for the method is an error.
+	if retErr := returnValues[0].Interface(); retErr != nil {
+		errMsg = retErr.(error).Error()
+	}
+
+	// write response
+	code := 200
+	rspReply := replyv.Interface()
+	if len(errMsg) != 0 {
+		code = 500
+		rspReply = invalidRequest
+	}
+	rspStream, err := codec.Write(errMsg, rspReply)
+	if err != nil {
+		return jerrors.Trace(err)
+	}
+	rsp := &http.Response{
+		StatusCode:    code,
+		ProtoMajor:    1,
+		ProtoMinor:    1,
+		Header:        make(http.Header),
+		ContentLength: int64(len(rspStream)),
+		Body:          ioutil.NopCloser(bytes.NewReader(rspStream)),
+	}
+	delete(header, "Content-Type")
+	delete(header, "Content-Length")
+	delete(header, "Timeout")
+	for k, v := range header {
+		rsp.Header.Set(k, v)
+	}
+
+	rspBuf := bytes.NewBuffer(make([]byte, DefaultHTTPRspBufferSize))
+	rspBuf.Reset()
+	if err = rsp.Write(rspBuf); err != nil {
+		log.Warn("rsp.Write(rsp:%#v) = error:%s", rsp, err)
+		return nil
+	}
+	if _, err = rspBuf.WriteTo(conn); err != nil {
+		log.Warn("rspBuf.WriteTo(conn:%#v) = error:%s", conn, err)
+	}
+	return nil
+}
diff --git a/protocol/protocol.go b/protocol/protocol.go
new file mode 100644
index 0000000000000000000000000000000000000000..413b2e61f693a000806c6837bc2cefd8a7c254fe
--- /dev/null
+++ b/protocol/protocol.go
@@ -0,0 +1,115 @@
+package protocol
+
+import (
+	"sync"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
+
+// Extension - protocol
+type Protocol interface {
+	Export(invoker Invoker) Exporter
+	Refer(url config.URL) Invoker
+	Destroy()
+}
+
+// wrapping invoker
+type Exporter interface {
+	GetInvoker() Invoker
+	Unexport()
+}
+
+/////////////////////////////
+// base protocol
+/////////////////////////////
+
+type BaseProtocol struct {
+	exporterMap *sync.Map
+	invokers    []Invoker
+}
+
+func NewBaseProtocol() BaseProtocol {
+	return BaseProtocol{
+		exporterMap: new(sync.Map),
+	}
+}
+
+func (bp *BaseProtocol) SetExporterMap(key string, exporter Exporter) {
+	bp.exporterMap.Store(key, exporter)
+}
+
+func (bp *BaseProtocol) ExporterMap() *sync.Map {
+	return bp.exporterMap
+}
+
+func (bp *BaseProtocol) SetInvokers(invoker Invoker) {
+	bp.invokers = append(bp.invokers, invoker)
+}
+
+func (bp *BaseProtocol) Invokers() []Invoker {
+	return bp.invokers
+}
+
+func (bp *BaseProtocol) Export(invoker Invoker) Exporter {
+	return nil
+}
+
+func (bp *BaseProtocol) Refer(url config.URL) Invoker {
+	return nil
+}
+
+// Destroy will destroy all invoker and exporter, so it only is called once.
+func (bp *BaseProtocol) Destroy() {
+	// destroy invokers
+	for _, invoker := range bp.invokers {
+		if invoker != nil {
+			invoker.Destroy()
+		}
+	}
+	bp.invokers = []Invoker{}
+
+	// unexport exporters
+	bp.exporterMap.Range(func(key, exporter interface{}) bool {
+		if exporter != nil {
+			exporter.(Exporter).Unexport()
+		} else {
+			bp.exporterMap.Delete(key)
+		}
+		return true
+	})
+}
+
+/////////////////////////////
+// base exporter
+/////////////////////////////
+
+type BaseExporter struct {
+	key         string
+	invoker     Invoker
+	exporterMap *sync.Map
+}
+
+func NewBaseExporter(key string, invoker Invoker, exporterMap *sync.Map) *BaseExporter {
+	return &BaseExporter{
+		key:         key,
+		invoker:     invoker,
+		exporterMap: exporterMap,
+	}
+}
+
+func (de *BaseExporter) GetInvoker() Invoker {
+	return de.invoker
+
+}
+
+func (de *BaseExporter) Unexport() {
+	log.Info("Exporter unexport.")
+	de.invoker.Destroy()
+	de.exporterMap.Delete(de.key)
+}
diff --git a/protocol/protocolwrapper/mock_protocol_filter.go b/protocol/protocolwrapper/mock_protocol_filter.go
new file mode 100644
index 0000000000000000000000000000000000000000..e2c0326f3814b7825f5aeb8abd953926cfecfc71
--- /dev/null
+++ b/protocol/protocolwrapper/mock_protocol_filter.go
@@ -0,0 +1,25 @@
+package protocolwrapper
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+type mockProtocolFilter struct {
+}
+
+func NewMockProtocolFilter() protocol.Protocol {
+	return &mockProtocolFilter{}
+}
+
+func (pfw *mockProtocolFilter) Export(invoker protocol.Invoker) protocol.Exporter {
+	return protocol.NewBaseExporter("key", invoker, nil)
+}
+
+func (pfw *mockProtocolFilter) Refer(url config.URL) protocol.Invoker {
+	return protocol.NewBaseInvoker(url)
+}
+
+func (pfw *mockProtocolFilter) Destroy() {
+
+}
diff --git a/protocol/protocolwrapper/protocol_filter_wrapper.go b/protocol/protocolwrapper/protocol_filter_wrapper.go
new file mode 100644
index 0000000000000000000000000000000000000000..8c7545b0765edbe628f534a9ca9de81159b4d6ee
--- /dev/null
+++ b/protocol/protocolwrapper/protocol_filter_wrapper.go
@@ -0,0 +1,94 @@
+package protocolwrapper
+
+import (
+	"strings"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/filter"
+	"github.com/dubbo/go-for-apache-dubbo/filter/imp"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+const FILTER = "filter"
+
+func init() {
+	extension.SetProtocol(FILTER, GetProtocol)
+}
+
+// protocol in url decide who ProtocolFilterWrapper.protocol is
+type ProtocolFilterWrapper struct {
+	protocol protocol.Protocol
+}
+
+func (pfw *ProtocolFilterWrapper) Export(invoker protocol.Invoker) protocol.Exporter {
+	if pfw.protocol == nil {
+		pfw.protocol = extension.GetProtocolExtension(invoker.GetUrl().Protocol)
+	}
+	invoker = buildInvokerChain(invoker, constant.SERVICE_FILTER_KEY)
+	return pfw.protocol.Export(invoker)
+}
+
+func (pfw *ProtocolFilterWrapper) Refer(url config.URL) protocol.Invoker {
+	if pfw.protocol == nil {
+		pfw.protocol = extension.GetProtocolExtension(url.Protocol)
+	}
+	return buildInvokerChain(pfw.protocol.Refer(url), constant.REFERENCE_FILTER_KEY)
+}
+
+func (pfw *ProtocolFilterWrapper) Destroy() {
+	pfw.protocol.Destroy()
+}
+
+func buildInvokerChain(invoker protocol.Invoker, key string) protocol.Invoker {
+	filtName := invoker.GetUrl().Params.Get(key)
+	if filtName == "" { // echo must be the first
+		filtName = imp.ECHO
+	} else {
+		filtName = imp.ECHO + "," + filtName
+	}
+	filtNames := strings.Split(filtName, ",")
+	next := invoker
+	// The order of filters is from left to right, so loading from right to left
+	for i := len(filtNames) - 1; i >= 0; i-- {
+		filter := extension.GetFilterExtension(filtNames[i])
+		fi := &FilterInvoker{next: next, invoker: invoker, filter: filter}
+		next = fi
+	}
+
+	return next
+}
+
+func GetProtocol() protocol.Protocol {
+	return &ProtocolFilterWrapper{}
+}
+
+///////////////////////////
+// filter invoker
+///////////////////////////
+
+type FilterInvoker struct {
+	next    protocol.Invoker
+	invoker protocol.Invoker
+	filter  filter.Filter
+}
+
+func (fi *FilterInvoker) GetUrl() config.URL {
+	return fi.invoker.GetUrl()
+}
+
+func (fi *FilterInvoker) IsAvailable() bool {
+	return fi.invoker.IsAvailable()
+}
+
+func (fi *FilterInvoker) Invoke(invocation protocol.Invocation) protocol.Result {
+	result := fi.filter.Invoke(fi.next, invocation)
+	return fi.filter.OnResponse(result, fi.invoker, invocation)
+}
+
+func (fi *FilterInvoker) Destroy() {
+	fi.invoker.Destroy()
+}
diff --git a/protocol/result.go b/protocol/result.go
new file mode 100644
index 0000000000000000000000000000000000000000..ba75e6178eeffb36d82213082bca9fe39768e7df
--- /dev/null
+++ b/protocol/result.go
@@ -0,0 +1,23 @@
+package protocol
+
+type Result interface {
+	Error() error
+	Result() interface{}
+}
+
+/////////////////////////////
+// Result Impletment of RPC
+/////////////////////////////
+
+type RPCResult struct {
+	Err  error
+	Rest interface{}
+}
+
+func (r *RPCResult) Error() error {
+	return r.Err
+}
+
+func (r *RPCResult) Result() interface{} {
+	return r.Rest
+}
diff --git a/protocol/support/rpcinvocation.go b/protocol/support/rpcinvocation.go
new file mode 100644
index 0000000000000000000000000000000000000000..f71e094d9d6678f43889dd23eeaec2f210b77f1d
--- /dev/null
+++ b/protocol/support/rpcinvocation.go
@@ -0,0 +1,106 @@
+package support
+
+import (
+	"reflect"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+)
+
+/////////////////////////////
+// Invocation Impletment of RPC
+/////////////////////////////
+// todo: is it necessary to separate fields of consumer(provider) from RPCInvocation
+type RPCInvocation struct {
+	methodName     string
+	parameterTypes []reflect.Type
+	arguments      []interface{}
+	reply          interface{}
+	callBack       interface{}
+	attachments    map[string]string
+	invoker        protocol.Invoker
+}
+
+func NewRPCInvocationForConsumer(methodName string, parameterTypes []reflect.Type, arguments []interface{},
+	reply interface{}, callBack interface{}, url config.URL, invoker protocol.Invoker) *RPCInvocation {
+
+	attachments := map[string]string{}
+	attachments[constant.PATH_KEY] = url.Path
+	attachments[constant.GROUP_KEY] = url.GetParam(constant.GROUP_KEY, "")
+	attachments[constant.INTERFACE_KEY] = url.GetParam(constant.INTERFACE_KEY, "")
+	attachments[constant.VERSION_KEY] = url.GetParam(constant.VERSION_KEY, constant.DEFAULT_VERSION)
+
+	return &RPCInvocation{
+		methodName:     methodName,
+		parameterTypes: parameterTypes,
+		arguments:      arguments,
+		reply:          reply,
+		callBack:       callBack,
+		attachments:    attachments,
+		invoker:        invoker,
+	}
+}
+
+func NewRPCInvocationForProvider(url config.URL) *RPCInvocation {
+	attachments := map[string]string{}
+	attachments[constant.PATH_KEY] = url.Path
+	attachments[constant.GROUP_KEY] = url.GetParam(constant.GROUP_KEY, "")
+	attachments[constant.INTERFACE_KEY] = url.GetParam(constant.INTERFACE_KEY, "")
+	attachments[constant.VERSION_KEY] = url.GetParam(constant.VERSION_KEY, constant.DEFAULT_VERSION)
+	return &RPCInvocation{
+		attachments: attachments,
+	}
+}
+
+func (r *RPCInvocation) MethodName() string {
+	return r.methodName
+}
+
+func (r *RPCInvocation) ParameterTypes() []reflect.Type {
+	return r.parameterTypes
+}
+
+func (r *RPCInvocation) Arguments() []interface{} {
+	return r.arguments
+}
+
+func (r *RPCInvocation) Reply() interface{} {
+	return r.reply
+}
+
+func (r *RPCInvocation) Attachments() map[string]string {
+	return r.attachments
+}
+
+func (r *RPCInvocation) AttachmentsByKey(key string, defaultValue string) string {
+	if r.attachments == nil {
+		return defaultValue
+	}
+	value, ok := r.attachments[key]
+	if ok {
+		return value
+	}
+	return defaultValue
+}
+
+func (r *RPCInvocation) SetAttachments(key string, value string) {
+	if r.attachments == nil {
+		r.attachments = make(map[string]string)
+	}
+	r.attachments[key] = value
+}
+
+func (r *RPCInvocation) Invoker() protocol.Invoker {
+	return r.invoker
+}
+
+func (r *RPCInvocation) SetInvoker() protocol.Invoker {
+	return r.invoker
+}
+
+func (r *RPCInvocation) CallBack() interface{} {
+	return r.callBack
+}
diff --git a/public/codec.go b/public/codec.go
deleted file mode 100644
index e88b9829c577dca11a7a3ffdc2e4d97536d45035..0000000000000000000000000000000000000000
--- a/public/codec.go
+++ /dev/null
@@ -1,44 +0,0 @@
-package public
-
-//////////////////////////////////////////
-// codec type
-//////////////////////////////////////////
-
-type CodecType int
-
-const (
-	CODECTYPE_UNKNOWN CodecType = iota
-	CODECTYPE_JSONRPC
-	CODECTYPE_DUBBO
-)
-
-var codecTypeStrings = [...]string{
-	"unknown",
-	"jsonrpc",
-	"dubbo",
-}
-
-func (c CodecType) String() string {
-	typ := CODECTYPE_UNKNOWN
-	switch c {
-	case CODECTYPE_JSONRPC:
-		typ = c
-	case CODECTYPE_DUBBO:
-		typ = c
-	}
-
-	return codecTypeStrings[typ]
-}
-
-func GetCodecType(t string) CodecType {
-	var typ = CODECTYPE_UNKNOWN
-
-	switch t {
-	case codecTypeStrings[CODECTYPE_JSONRPC]:
-		typ = CODECTYPE_JSONRPC
-	case codecTypeStrings[CODECTYPE_DUBBO]:
-		typ = CODECTYPE_DUBBO
-	}
-
-	return typ
-}
diff --git a/public/const.go b/public/const.go
deleted file mode 100644
index d7f385ecf4530ebc2174c7162a44a88e65106319..0000000000000000000000000000000000000000
--- a/public/const.go
+++ /dev/null
@@ -1,5 +0,0 @@
-package public
-
-const (
-	DUBBOGO_CTX_KEY = "dubbogo-ctx"
-)
diff --git a/registry/directory/directory.go b/registry/directory/directory.go
new file mode 100644
index 0000000000000000000000000000000000000000..0abec42657f1b39fdc0576e153b9cd47c46c2cfc
--- /dev/null
+++ b/registry/directory/directory.go
@@ -0,0 +1,248 @@
+package directory
+
+import (
+	"sync"
+	"time"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+	jerrors "github.com/juju/errors"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster/directory"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/protocolwrapper"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
+)
+
+const RegistryConnDelay = 3
+
+type Options struct {
+	serviceTTL time.Duration
+}
+type Option func(*Options)
+
+type RegistryDirectory struct {
+	directory.BaseDirectory
+	cacheInvokers    []protocol.Invoker
+	listenerLock     sync.Mutex
+	serviceType      string
+	registry         registry.Registry
+	cacheInvokersMap *sync.Map //use sync.map
+	//cacheInvokersMap map[string]protocol.Invoker
+	Options
+}
+
+func NewRegistryDirectory(url *config.URL, registry registry.Registry, opts ...Option) (*RegistryDirectory, error) {
+	options := Options{
+		//default 300s
+		serviceTTL: time.Duration(300e9),
+	}
+	for _, opt := range opts {
+		opt(&options)
+	}
+	if url.SubURL == nil {
+		return nil, jerrors.Errorf("url is invalid, suburl can not be nil")
+	}
+	return &RegistryDirectory{
+		BaseDirectory:    directory.NewBaseDirectory(url),
+		cacheInvokers:    []protocol.Invoker{},
+		cacheInvokersMap: &sync.Map{},
+		serviceType:      url.SubURL.Service(),
+		registry:         registry,
+		Options:          options,
+	}, nil
+}
+
+//subscibe from registry
+func (dir *RegistryDirectory) Subscribe(url config.URL) {
+	for {
+		if !dir.registry.IsAvailable() {
+			log.Warn("event listener game over.")
+			return
+		}
+
+		listener, err := dir.registry.Subscribe(url)
+		if err != nil {
+			if !dir.registry.IsAvailable() {
+				log.Warn("event listener game over.")
+				return
+			}
+			log.Warn("getListener() = err:%s", jerrors.ErrorStack(err))
+			time.Sleep(time.Duration(RegistryConnDelay) * time.Second)
+			continue
+		}
+
+		for {
+			if serviceEvent, err := listener.Next(); err != nil {
+				log.Warn("Selector.watch() = error{%v}", jerrors.ErrorStack(err))
+				listener.Close()
+				time.Sleep(time.Duration(RegistryConnDelay) * time.Second)
+				return
+			} else {
+				log.Info("update begin, service event: %v", serviceEvent.String())
+				go dir.update(serviceEvent)
+			}
+
+		}
+
+	}
+}
+
+//subscribe service from registry , and update the cacheServices
+func (dir *RegistryDirectory) update(res *registry.ServiceEvent) {
+	if res == nil {
+		return
+	}
+
+	log.Debug("registry update, result{%s}", res)
+
+	log.Debug("update service name: %s!", res.Service)
+
+	dir.refreshInvokers(res)
+}
+
+func (dir *RegistryDirectory) refreshInvokers(res *registry.ServiceEvent) {
+	var newCacheInvokersMap sync.Map
+
+	switch res.Action {
+	case registry.ServiceAdd:
+		//dir.cacheService.Add(res.Path, dir.serviceTTL)
+		newCacheInvokersMap = *dir.cacheInvoker(res.Service)
+	case registry.ServiceDel:
+		//dir.cacheService.Del(res.Path, dir.serviceTTL)
+		newCacheInvokersMap = *dir.uncacheInvoker(res.Service)
+		log.Info("selector delete service url{%s}", res.Service)
+	default:
+		return
+	}
+
+	newInvokers := dir.toGroupInvokers(&newCacheInvokersMap)
+
+	dir.listenerLock.Lock()
+	defer dir.listenerLock.Unlock()
+	dir.cacheInvokers = newInvokers
+}
+
+func (dir *RegistryDirectory) toGroupInvokers(newInvokersMap *sync.Map) []protocol.Invoker {
+
+	newInvokersList := []protocol.Invoker{}
+	groupInvokersMap := make(map[string][]protocol.Invoker)
+	groupInvokersList := []protocol.Invoker{}
+
+	newInvokersMap.Range(func(key, value interface{}) bool {
+		newInvokersList = append(newInvokersList, value.(protocol.Invoker))
+		return true
+	})
+
+	for _, invoker := range newInvokersList {
+		group := invoker.GetUrl().GetParam(constant.GROUP_KEY, "")
+
+		if _, ok := groupInvokersMap[group]; ok {
+			groupInvokersMap[group] = append(groupInvokersMap[group], invoker)
+		} else {
+			groupInvokersMap[group] = []protocol.Invoker{invoker}
+		}
+	}
+	if len(groupInvokersMap) == 1 {
+		//len is 1 it means no group setting ,so do not need cluster again
+		groupInvokersList = groupInvokersMap[""]
+	} else {
+		for _, invokers := range groupInvokersMap {
+			staticDir := directory.NewStaticDirectory(invokers)
+			cluster := extension.GetCluster(dir.GetUrl().SubURL.GetParam(constant.CLUSTER_KEY, constant.DEFAULT_CLUSTER))
+			groupInvokersList = append(groupInvokersList, cluster.Join(staticDir))
+		}
+	}
+
+	return groupInvokersList
+}
+
+func (dir *RegistryDirectory) uncacheInvoker(url config.URL) *sync.Map {
+	log.Debug("service will be deleted in cache invokers: invokers key is  %s!", url.Key())
+	newCacheInvokers := dir.cacheInvokersMap
+	newCacheInvokers.Delete(url.Key())
+	return newCacheInvokers
+}
+
+func (dir *RegistryDirectory) cacheInvoker(url config.URL) *sync.Map {
+	referenceUrl := dir.GetUrl().SubURL
+	newCacheInvokers := dir.cacheInvokersMap
+	//check the url's protocol is equal to the protocol which is configured in reference config or referenceUrl is not care about protocol
+	if url.Protocol == referenceUrl.Protocol || referenceUrl.Protocol == "" {
+		url = mergeUrl(url, referenceUrl)
+
+		if _, ok := newCacheInvokers.Load(url.Key()); !ok {
+			log.Debug("service will be added in cache invokers: invokers key is  %s!", url.Key())
+			newInvoker := extension.GetProtocolExtension(protocolwrapper.FILTER).Refer(url)
+			if newInvoker != nil {
+				newCacheInvokers.Store(url.Key(), newInvoker)
+			}
+		}
+	}
+	return newCacheInvokers
+}
+
+//select the protocol invokers from the directory
+func (dir *RegistryDirectory) List(invocation protocol.Invocation) []protocol.Invoker {
+	//TODO:router
+	return dir.cacheInvokers
+}
+
+func (dir *RegistryDirectory) IsAvailable() bool {
+	return dir.BaseDirectory.IsAvailable()
+}
+
+func (dir *RegistryDirectory) Destroy() {
+	//dir.registry.Destroy() should move it in protocol
+	//TODO:unregister & unsubscribe
+	dir.BaseDirectory.Destroy()
+}
+
+// configuration  > reference config >service config
+//  in this function we should merge the reference local url config into the service url from registry.
+//TODO configuration merge, in the future , the configuration center's config should merge too.
+func mergeUrl(serviceUrl config.URL, referenceUrl *config.URL) config.URL {
+	mergedUrl := serviceUrl
+	var methodConfigMergeFcn = []func(method string){}
+
+	//loadBalance strategy config
+	if v := referenceUrl.Params.Get(constant.LOADBALANCE_KEY); v != "" {
+		mergedUrl.Params.Set(constant.LOADBALANCE_KEY, v)
+	}
+	methodConfigMergeFcn = append(methodConfigMergeFcn, func(method string) {
+		if v := referenceUrl.Params.Get(method + "." + constant.LOADBALANCE_KEY); v != "" {
+			mergedUrl.Params.Set(method+"."+constant.LOADBALANCE_KEY, v)
+		}
+	})
+
+	//cluster strategy config
+	if v := referenceUrl.Params.Get(constant.CLUSTER_KEY); v != "" {
+		mergedUrl.Params.Set(constant.CLUSTER_KEY, v)
+	}
+	methodConfigMergeFcn = append(methodConfigMergeFcn, func(method string) {
+		if v := referenceUrl.Params.Get(method + "." + constant.CLUSTER_KEY); v != "" {
+			mergedUrl.Params.Set(method+"."+constant.CLUSTER_KEY, v)
+		}
+	})
+
+	//remote timestamp
+	if v := serviceUrl.Params.Get(constant.TIMESTAMP_KEY); v != "" {
+		mergedUrl.Params.Set(constant.REMOTE_TIMESTAMP_KEY, v)
+		mergedUrl.Params.Set(constant.TIMESTAMP_KEY, referenceUrl.Params.Get(constant.TIMESTAMP_KEY))
+	}
+
+	//finally execute methodConfigMergeFcn
+	for _, method := range referenceUrl.Methods {
+		for _, fcn := range methodConfigMergeFcn {
+			fcn("methods." + method)
+		}
+	}
+
+	return mergedUrl
+}
diff --git a/registry/directory/directory_test.go b/registry/directory/directory_test.go
new file mode 100644
index 0000000000000000000000000000000000000000..8baf6905afafef4417e4792fd5bc842721747112
--- /dev/null
+++ b/registry/directory/directory_test.go
@@ -0,0 +1,99 @@
+package directory
+
+import (
+	"context"
+	"net/url"
+	"strconv"
+	"testing"
+	"time"
+)
+import (
+	"github.com/stretchr/testify/assert"
+)
+import (
+	"github.com/dubbo/go-for-apache-dubbo/cluster/support"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/protocolwrapper"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
+)
+
+func TestSubscribe(t *testing.T) {
+	extension.SetProtocol(protocolwrapper.FILTER, protocolwrapper.NewMockProtocolFilter)
+
+	url, _ := config.NewURL(context.TODO(), "mock://127.0.0.1:1111")
+	suburl, _ := config.NewURL(context.TODO(), "dubbo://127.0.0.1:20000")
+	url.SubURL = &suburl
+	mockRegistry := registry.NewMockRegistry()
+	registryDirectory, _ := NewRegistryDirectory(&url, mockRegistry)
+
+	go registryDirectory.Subscribe(*config.NewURLWithOptions("testservice"))
+	for i := 0; i < 3; i++ {
+		mockRegistry.MockEvent(&registry.ServiceEvent{Action: registry.ServiceAdd, Service: *config.NewURLWithOptions("TEST"+strconv.FormatInt(int64(i), 10), config.WithProtocol("dubbo"))})
+	}
+
+	time.Sleep(1e9)
+	assert.Len(t, registryDirectory.cacheInvokers, 3)
+}
+
+func TestSubscribe_Delete(t *testing.T) {
+	extension.SetProtocol(protocolwrapper.FILTER, protocolwrapper.NewMockProtocolFilter)
+
+	url, _ := config.NewURL(context.TODO(), "mock://127.0.0.1:1111")
+	suburl, _ := config.NewURL(context.TODO(), "dubbo://127.0.0.1:20000")
+	url.SubURL = &suburl
+	mockRegistry := registry.NewMockRegistry()
+	registryDirectory, _ := NewRegistryDirectory(&url, mockRegistry)
+
+	go registryDirectory.Subscribe(*config.NewURLWithOptions("testservice"))
+	for i := 0; i < 3; i++ {
+		mockRegistry.MockEvent(&registry.ServiceEvent{Action: registry.ServiceAdd, Service: *config.NewURLWithOptions("TEST"+strconv.FormatInt(int64(i), 10), config.WithProtocol("dubbo"))})
+	}
+	time.Sleep(1e9)
+	assert.Len(t, registryDirectory.cacheInvokers, 3)
+	mockRegistry.MockEvent(&registry.ServiceEvent{Action: registry.ServiceDel, Service: *config.NewURLWithOptions("TEST0", config.WithProtocol("dubbo"))})
+	time.Sleep(1e9)
+	assert.Len(t, registryDirectory.cacheInvokers, 2)
+
+}
+func TestSubscribe_InvalidUrl(t *testing.T) {
+	url, _ := config.NewURL(context.TODO(), "mock://127.0.0.1:1111")
+	mockRegistry := registry.NewMockRegistry()
+	_, err := NewRegistryDirectory(&url, mockRegistry)
+	assert.Error(t, err)
+}
+
+func TestSubscribe_Group(t *testing.T) {
+	extension.SetProtocol(protocolwrapper.FILTER, protocolwrapper.NewMockProtocolFilter)
+	extension.SetCluster("mock", cluster.NewMockCluster)
+
+	regurl, _ := config.NewURL(context.TODO(), "mock://127.0.0.1:1111")
+	suburl, _ := config.NewURL(context.TODO(), "dubbo://127.0.0.1:20000")
+	suburl.Params.Set(constant.CLUSTER_KEY, "mock")
+	regurl.SubURL = &suburl
+	mockRegistry := registry.NewMockRegistry()
+	registryDirectory, _ := NewRegistryDirectory(&regurl, mockRegistry)
+
+	go registryDirectory.Subscribe(*config.NewURLWithOptions("testservice"))
+
+	//for group1
+	urlmap := url.Values{}
+	urlmap.Set(constant.GROUP_KEY, "group1")
+	urlmap.Set(constant.CLUSTER_KEY, "failover") //to test merge url
+	for i := 0; i < 3; i++ {
+		mockRegistry.MockEvent(&registry.ServiceEvent{Action: registry.ServiceAdd, Service: *config.NewURLWithOptions("TEST"+strconv.FormatInt(int64(i), 10), config.WithProtocol("dubbo"),
+			config.WithParams(urlmap))})
+	}
+	//for group2
+	urlmap2 := url.Values{}
+	urlmap2.Set(constant.GROUP_KEY, "group2")
+	urlmap2.Set(constant.CLUSTER_KEY, "failover") //to test merge url
+	for i := 0; i < 3; i++ {
+		mockRegistry.MockEvent(&registry.ServiceEvent{Action: registry.ServiceAdd, Service: *config.NewURLWithOptions("TEST"+strconv.FormatInt(int64(i), 10), config.WithProtocol("dubbo"),
+			config.WithParams(urlmap2))})
+	}
+
+	time.Sleep(1e9)
+	assert.Len(t, registryDirectory.cacheInvokers, 2)
+}
diff --git a/registry/event.go b/registry/event.go
index 13bb4f73fd0e3870706523e2e7a44360fbc7aa22..976f7f9875ce4da864c2cc3775a5dcd3df060238 100644
--- a/registry/event.go
+++ b/registry/event.go
@@ -5,6 +5,9 @@ import (
 	"math/rand"
 	"time"
 )
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
 
 func init() {
 	rand.Seed(time.Now().UnixNano())
@@ -36,9 +39,9 @@ func (t ServiceEventType) String() string {
 
 type ServiceEvent struct {
 	Action  ServiceEventType
-	Service ServiceURL
+	Service config.URL
 }
 
 func (e ServiceEvent) String() string {
-	return fmt.Sprintf("ServiceEvent{Action{%s}, Service{%s}}", e.Action, e.Service)
+	return fmt.Sprintf("ServiceEvent{Action{%s}, Path{%s}}", e.Action, e.Service)
 }
diff --git a/registry/factory.go b/registry/factory.go
new file mode 100644
index 0000000000000000000000000000000000000000..bb20e9dd3ebcd83532ac57e178b3e73414090b5d
--- /dev/null
+++ b/registry/factory.go
@@ -0,0 +1,7 @@
+package registry
+
+import "github.com/dubbo/go-for-apache-dubbo/config"
+
+type RegistryFactory interface {
+	GetRegistry(url config.URL) Registry
+}
diff --git a/registry/mock_registry.go b/registry/mock_registry.go
new file mode 100644
index 0000000000000000000000000000000000000000..9036d9a93f6118999ffa98439f65a5dc5dc7843f
--- /dev/null
+++ b/registry/mock_registry.go
@@ -0,0 +1,59 @@
+package registry
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/tevino/abool"
+)
+
+type MockRegistry struct {
+	listener  *listener
+	destroyed *abool.AtomicBool
+}
+
+func NewMockRegistry() *MockRegistry {
+	registry := &MockRegistry{
+		destroyed: abool.NewBool(false),
+	}
+	listener := &listener{count: 0, registry: registry, listenChan: make(chan *ServiceEvent)}
+	registry.listener = listener
+	return registry
+}
+func (*MockRegistry) Register(url config.URL) error {
+	return nil
+}
+
+func (r *MockRegistry) Destroy() {
+	if r.destroyed.SetToIf(false, true) {
+	}
+}
+func (r *MockRegistry) IsAvailable() bool {
+	return !r.destroyed.IsSet()
+}
+func (r *MockRegistry) GetUrl() config.URL {
+	return config.URL{}
+}
+
+func (r *MockRegistry) Subscribe(config.URL) (Listener, error) {
+	return r.listener, nil
+}
+
+type listener struct {
+	count      int64
+	registry   *MockRegistry
+	listenChan chan *ServiceEvent
+}
+
+func (l *listener) Next() (*ServiceEvent, error) {
+	select {
+	case e := <-l.listenChan:
+		return e, nil
+	}
+}
+
+func (*listener) Close() {
+
+}
+
+func (r *MockRegistry) MockEvent(event *ServiceEvent) {
+	r.listener.listenChan <- event
+}
diff --git a/registry/options.go b/registry/options.go
deleted file mode 100644
index 28197a7af04972c0893d3ee2d685d9b1eaab77b2..0000000000000000000000000000000000000000
--- a/registry/options.go
+++ /dev/null
@@ -1,76 +0,0 @@
-package registry
-
-import (
-	"fmt"
-)
-
-/////////////////////////////////
-// dubbo role type
-/////////////////////////////////
-
-const (
-	CONSUMER = iota
-	CONFIGURATOR
-	ROUTER
-	PROVIDER
-)
-
-var (
-	DubboNodes = [...]string{"consumers", "configurators", "routers", "providers"}
-	DubboRole  = [...]string{"consumer", "", "", "provider"}
-)
-
-type DubboType int
-
-func (t DubboType) String() string {
-	return DubboNodes[t]
-}
-
-func (t DubboType) Role() string {
-	return DubboRole[t]
-}
-
-/////////////////////////////////
-// dubbo config & options
-/////////////////////////////////
-
-type RegistryOption interface {
-	Name() string
-}
-
-type ApplicationConfig struct {
-	Organization string `yaml:"organization"  json:"organization,omitempty"`
-	Name         string `yaml:"name" json:"name,omitempty"`
-	Module       string `yaml:"module" json:"module,omitempty"`
-	Version      string `yaml:"version" json:"version,omitempty"`
-	Owner        string `yaml:"owner" json:"owner,omitempty"`
-	Environment  string `yaml:"environment" json:"environment,omitempty"`
-}
-
-type Options struct {
-	ApplicationConfig
-	DubboType DubboType
-}
-
-func (o *Options) String() string {
-	return fmt.Sprintf("name:%s, version:%s, owner:%s, module:%s, organization:%s, type:%s",
-		o.Name, o.Version, o.Owner, o.Module, o.Organization, o.DubboType)
-}
-
-type Option func(*Options)
-
-func (Option) Name() string {
-	return "dubbogo-registry-option"
-}
-
-func WithDubboType(typ DubboType) Option {
-	return func(o *Options) {
-		o.DubboType = typ
-	}
-}
-
-func WithApplicationConf(conf ApplicationConfig) Option {
-	return func(o *Options) {
-		o.ApplicationConfig = conf
-	}
-}
diff --git a/registry/protocol/protocol.go b/registry/protocol/protocol.go
new file mode 100644
index 0000000000000000000000000000000000000000..c982edcb81d35a474e742d2c21962a30be46d702
--- /dev/null
+++ b/registry/protocol/protocol.go
@@ -0,0 +1,164 @@
+package protocol
+
+import (
+	"sync"
+)
+
+import (
+	log "github.com/AlexStocks/log4go"
+)
+
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/protocol"
+	"github.com/dubbo/go-for-apache-dubbo/protocol/protocolwrapper"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
+	directory2 "github.com/dubbo/go-for-apache-dubbo/registry/directory"
+)
+
+var registryProtocol *RegistryProtocol
+
+type RegistryProtocol struct {
+	// Registry  Map<RegistryAddress, Registry>
+	registries sync.Map
+	//To solve the problem of RMI repeated exposure port conflicts, the services that have been exposed are no longer exposed.
+	//providerurl <--> exporter
+	bounds sync.Map
+}
+
+func init() {
+	extension.SetProtocol("registry", GetProtocol)
+}
+
+func NewRegistryProtocol() *RegistryProtocol {
+	return &RegistryProtocol{
+		registries: sync.Map{},
+		bounds:     sync.Map{},
+	}
+}
+func getRegistry(regUrl *config.URL) registry.Registry {
+	reg, err := extension.GetRegistryExtension(regUrl.Protocol, regUrl)
+	if err != nil {
+		log.Error("Registry can not connect success, program is going to panic.Error message is %s", err.Error())
+		panic(err.Error())
+	}
+	return reg
+}
+func (proto *RegistryProtocol) Refer(url config.URL) protocol.Invoker {
+
+	var registryUrl = url
+	var serviceUrl = registryUrl.SubURL
+	if registryUrl.Protocol == constant.REGISTRY_PROTOCOL {
+		protocol := registryUrl.GetParam(constant.REGISTRY_KEY, "")
+		registryUrl.Protocol = protocol
+	}
+	var reg registry.Registry
+
+	if regI, loaded := proto.registries.Load(registryUrl.Key()); !loaded {
+		reg = getRegistry(&registryUrl)
+		proto.registries.Store(registryUrl.Key(), reg)
+	} else {
+		reg = regI.(registry.Registry)
+	}
+
+	//new registry directory for store service url from registry
+	directory, err := directory2.NewRegistryDirectory(&registryUrl, reg)
+	if err != nil {
+		log.Error("consumer service %v  create registry directory  error, error message is %s, and will return nil invoker!", serviceUrl.String(), err.Error())
+		return nil
+	}
+	err = reg.Register(*serviceUrl)
+	if err != nil {
+		log.Error("consumer service %v register registry %v error, error message is %s", serviceUrl.String(), registryUrl.String(), err.Error())
+	}
+	go directory.Subscribe(*serviceUrl)
+
+	//new cluster invoker
+	cluster := extension.GetCluster(serviceUrl.Params.Get(constant.CLUSTER_KEY))
+
+	return cluster.Join(directory)
+}
+
+func (proto *RegistryProtocol) Export(invoker protocol.Invoker) protocol.Exporter {
+	registryUrl := proto.getRegistryUrl(invoker)
+	providerUrl := proto.getProviderUrl(invoker)
+
+	var reg registry.Registry
+
+	if regI, loaded := proto.registries.Load(registryUrl.Key()); !loaded {
+		reg = getRegistry(&registryUrl)
+		proto.registries.Store(registryUrl.Key(), reg)
+	} else {
+		reg = regI.(registry.Registry)
+	}
+
+	err := reg.Register(providerUrl)
+	if err != nil {
+		log.Error("provider service %v register registry %v error, error message is %s", providerUrl.Key(), registryUrl.Key(), err.Error())
+		return nil
+	}
+
+	key := providerUrl.Key()
+	log.Info("The cached exporter keys is %v !", key)
+	cachedExporter, loaded := proto.bounds.Load(key)
+	if loaded {
+		log.Info("The exporter has been cached, and will return cached exporter!")
+	} else {
+		wrappedInvoker := newWrappedInvoker(invoker, providerUrl)
+		cachedExporter = extension.GetProtocolExtension(protocolwrapper.FILTER).Export(wrappedInvoker)
+		proto.bounds.Store(key, cachedExporter)
+		log.Info("The exporter has not been cached, and will return a new  exporter!")
+	}
+
+	return cachedExporter.(protocol.Exporter)
+
+}
+
+func (*RegistryProtocol) Destroy() {
+
+}
+
+func (*RegistryProtocol) getRegistryUrl(invoker protocol.Invoker) config.URL {
+	//here add * for return a new url
+	url := invoker.GetUrl()
+	//if the protocol == registry ,set protocol the registry value in url.params
+	if url.Protocol == constant.REGISTRY_PROTOCOL {
+		protocol := url.GetParam(constant.REGISTRY_KEY, "")
+		url.Protocol = protocol
+	}
+	return url
+}
+
+func (*RegistryProtocol) getProviderUrl(invoker protocol.Invoker) config.URL {
+	url := invoker.GetUrl()
+	return *url.SubURL
+}
+
+func GetProtocol() protocol.Protocol {
+	if registryProtocol != nil {
+		return registryProtocol
+	}
+	return NewRegistryProtocol()
+}
+
+type wrappedInvoker struct {
+	invoker protocol.Invoker
+	url     config.URL
+	protocol.BaseInvoker
+}
+
+func newWrappedInvoker(invoker protocol.Invoker, url config.URL) *wrappedInvoker {
+	return &wrappedInvoker{
+		invoker:     invoker,
+		url:         url,
+		BaseInvoker: *protocol.NewBaseInvoker(config.URL{}),
+	}
+}
+func (ivk *wrappedInvoker) GetUrl() config.URL {
+	return ivk.url
+}
+func (ivk *wrappedInvoker) getInvoker() protocol.Invoker {
+	return ivk.invoker
+}
diff --git a/registry/registry.go b/registry/registry.go
index 8408b14e0a2cf5339901d0b954b6854f9d01233a..e8711f7fc57c4c1b4940d630e92c0aad6c73014e 100644
--- a/registry/registry.go
+++ b/registry/registry.go
@@ -1,25 +1,26 @@
 package registry
 
-//////////////////////////////////////////////
-// Registry Interface
-//////////////////////////////////////////////
+import (
+	"github.com/dubbo/go-for-apache-dubbo/common"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+)
 
-// for service discovery/registry
+// Extension - Registry
 type Registry interface {
-
+	common.Node
 	//used for service provider calling , register services to registry
 	//And it is also used for service consumer calling , register services cared about ,for dubbo's admin monitoring.
-	Register(ServiceConfig) error
+	Register(url config.URL) error
 
 	//used for service consumer ,start subscribe service event from registry
-	Subscribe() (Listener, error)
+	Subscribe(config.URL) (Listener, error)
 
 	//input the serviceConfig , registry should return serviceUrlArray with multi location(provider nodes) available
-	GetService(ServiceConfig) ([]ServiceURL, error)
+	//GetService(SubURL) ([]SubURL, error)
 	//close the registry for Elegant closing
-	Close()
+	//Close()
 	//return if the registry is closed for consumer subscribing
-	IsClosed() bool
+	//IsClosed() bool
 }
 
 type Listener interface {
diff --git a/registry/service.go b/registry/service.go
deleted file mode 100644
index 7d89796aed1a1bb21ab297b553e75123c9e494cc..0000000000000000000000000000000000000000
--- a/registry/service.go
+++ /dev/null
@@ -1,293 +0,0 @@
-package registry
-
-import (
-	"fmt"
-	"net"
-	"net/url"
-	"strconv"
-	"strings"
-	"time"
-)
-
-import (
-	jerrors "github.com/juju/errors"
-)
-
-//////////////////////////////////////////////
-// service config
-//////////////////////////////////////////////
-
-type ServiceConfig interface {
-	Key() string
-	String() string
-	ServiceEqual(url ServiceURL) bool
-	//your service config implements must contain properties below
-	Service() string
-	Protocol() string
-	Version() string
-	Group() string
-	SetProtocol(string)
-	SetService(string)
-	SetVersion(string)
-	SetGroup(string)
-}
-
-type ProviderServiceConfig interface {
-	//your service config implements must contain properties below
-	ServiceConfig
-	Methods() string
-	Path() string
-	SetMethods(string)
-	SetPath(string)
-}
-
-type DefaultServiceConfig struct {
-	Protocol_ string `required:"true",default:"dubbo"  yaml:"protocol"  json:"protocol,omitempty"`
-	Service_  string `required:"true"  yaml:"service"  json:"service,omitempty"`
-	Group_    string `yaml:"group" json:"group,omitempty"`
-	Version_  string `yaml:"version" json:"version,omitempty"`
-}
-
-func NewDefaultServiceConfig() ServiceConfig {
-	return &DefaultServiceConfig{}
-}
-
-func (c *DefaultServiceConfig) Key() string {
-	return fmt.Sprintf("%s@%s", c.Service_, c.Protocol_)
-}
-
-func (c *DefaultServiceConfig) String() string {
-	return fmt.Sprintf("%s@%s-%s-%s", c.Service_, c.Protocol_, c.Group_, c.Version_)
-}
-
-func (c *DefaultServiceConfig) ServiceEqual(url ServiceURL) bool {
-	if c.Protocol_ != url.Protocol() {
-		return false
-	}
-
-	if c.Service_ != url.Query().Get("interface") {
-		return false
-	}
-
-	if c.Group_ != url.Group() {
-		return false
-	}
-
-	if c.Version_ != url.Version() {
-		return false
-	}
-
-	return true
-}
-
-func (c *DefaultServiceConfig) Service() string {
-	return c.Service_
-}
-
-func (c *DefaultServiceConfig) Protocol() string {
-	return c.Protocol_
-}
-
-func (c *DefaultServiceConfig) Version() string {
-	return c.Version_
-}
-
-func (c *DefaultServiceConfig) Group() string {
-	return c.Group_
-}
-func (c *DefaultServiceConfig) SetProtocol(s string) {
-	c.Protocol_ = s
-}
-
-func (c *DefaultServiceConfig) SetService(s string) {
-	c.Service_ = s
-}
-func (c *DefaultServiceConfig) SetVersion(s string) {
-	c.Version_ = s
-}
-
-func (c *DefaultServiceConfig) SetGroup(s string) {
-	c.Group_ = s
-}
-
-type DefaultProviderServiceConfig struct {
-	*DefaultServiceConfig
-	Path_    string `yaml:"path" json:"path,omitempty"`
-	Methods_ string `yaml:"methods" json:"methods,omitempty"`
-}
-
-func NewDefaultProviderServiceConfig() ProviderServiceConfig {
-	return &DefaultProviderServiceConfig{
-		DefaultServiceConfig: NewDefaultServiceConfig().(*DefaultServiceConfig),
-	}
-}
-
-func (c *DefaultProviderServiceConfig) Methods() string {
-	return c.Methods_
-}
-
-func (c *DefaultProviderServiceConfig) Path() string {
-	return c.Path_
-}
-
-func (c *DefaultProviderServiceConfig) SetMethods(s string) {
-	c.Methods_ = s
-}
-
-func (c *DefaultProviderServiceConfig) SetPath(s string) {
-	c.Path_ = s
-}
-
-//////////////////////////////////////////
-// service url
-//////////////////////////////////////////
-
-type ServiceURL interface {
-	ServiceConfig() ServiceConfig
-	CheckMethod(string) bool
-	PrimitiveURL() string
-	Query() url.Values
-	Location() string
-	Timeout() time.Duration
-	Group() string
-	Protocol() string
-	Version() string
-	Ip() string
-	Port() string
-	Path() string
-}
-
-type DefaultServiceURL struct {
-	Protocol_     string
-	Location_     string // ip+port
-	Path_         string // like  /com.ikurento.dubbo.UserProvider3
-	Ip_           string
-	Port_         string
-	Timeout_      time.Duration
-	Version_      string
-	Group_        string
-	Query_        url.Values
-	Weight_       int32
-	PrimitiveURL_ string
-}
-
-func NewDefaultServiceURL(urlString string) (ServiceURL, error) {
-	var (
-		err          error
-		rawUrlString string
-		serviceUrl   *url.URL
-		s            = &DefaultServiceURL{}
-	)
-
-	rawUrlString, err = url.QueryUnescape(urlString)
-	if err != nil {
-		return nil, jerrors.Errorf("url.QueryUnescape(%s),  error{%v}", urlString, err)
-	}
-
-	serviceUrl, err = url.Parse(rawUrlString)
-	if err != nil {
-		return nil, jerrors.Errorf("url.Parse(url string{%s}),  error{%v}", rawUrlString, err)
-	}
-
-	s.Query_, err = url.ParseQuery(serviceUrl.RawQuery)
-	if err != nil {
-		return nil, jerrors.Errorf("url.ParseQuery(raw url string{%s}),  error{%v}", serviceUrl.RawQuery, err)
-	}
-
-	s.PrimitiveURL_ = urlString
-	s.Protocol_ = serviceUrl.Scheme
-	s.Location_ = serviceUrl.Host
-	s.Path_ = serviceUrl.Path
-	if strings.Contains(s.Location_, ":") {
-		s.Ip_, s.Port_, err = net.SplitHostPort(s.Location_)
-		if err != nil {
-			return nil, jerrors.Errorf("net.SplitHostPort(Url.Host{%s}), error{%v}", s.Location_, err)
-		}
-	}
-	s.Group_ = s.Query_.Get("group")
-	s.Version_ = s.Query_.Get("version")
-	timeoutStr := s.Query_.Get("timeout")
-	if len(timeoutStr) == 0 {
-		timeoutStr = s.Query_.Get("default.timeout")
-	}
-	if len(timeoutStr) != 0 {
-		timeout, err := strconv.Atoi(timeoutStr)
-		if err == nil && timeout != 0 {
-			s.Timeout_ = time.Duration(timeout * 1e6) // timeout unit is millisecond
-		}
-	}
-
-	return s, nil
-}
-
-func (s DefaultServiceURL) String() string {
-	return fmt.Sprintf(
-		"DefaultServiceURL{Protocol:%s, Location:%s, Path:%s, Ip:%s, Port:%s, "+
-			"Timeout:%s, Version:%s, Group:%s, Weight_:%d, Query:%+v}",
-		s.Protocol_, s.Location_, s.Path_, s.Ip_, s.Port_,
-		s.Timeout_, s.Version_, s.Group_, s.Weight_, s.Query_)
-}
-
-func (s *DefaultServiceURL) ServiceConfig() ServiceConfig {
-	interfaceName := s.Query_.Get("interface")
-	return &DefaultServiceConfig{
-		Protocol_: s.Protocol_,
-		Service_:  interfaceName,
-		Group_:    s.Group_,
-		Version_:  s.Version_,
-	}
-}
-
-func (s *DefaultServiceURL) CheckMethod(method string) bool {
-	var (
-		methodArray []string
-	)
-
-	methodArray = strings.Split(s.Query_.Get("methods"), ",")
-	for _, m := range methodArray {
-		if m == method {
-			return true
-		}
-	}
-
-	return false
-}
-
-func (s *DefaultServiceURL) PrimitiveURL() string {
-	return s.PrimitiveURL_
-}
-
-func (s *DefaultServiceURL) Timeout() time.Duration {
-	return s.Timeout_
-}
-func (s *DefaultServiceURL) Location() string {
-	return s.Location_
-}
-
-func (s *DefaultServiceURL) Query() url.Values {
-	return s.Query_
-}
-
-func (s *DefaultServiceURL) Group() string {
-	return s.Group_
-}
-
-func (s *DefaultServiceURL) Protocol() string {
-	return s.Protocol_
-}
-
-func (s *DefaultServiceURL) Version() string {
-	return s.Version_
-}
-
-func (s *DefaultServiceURL) Ip() string {
-	return s.Ip_
-}
-
-func (s *DefaultServiceURL) Port() string {
-	return s.Port_
-}
-
-func (s *DefaultServiceURL) Path() string {
-	return s.Path_
-}
diff --git a/registry/zookeeper/consumer.go b/registry/zookeeper/consumer.go
index c1bf9ff32c2244912fc813049ca160ebe83deca0..88c2c6cd31d00e2622072fc47e583be09c471a4f 100644
--- a/registry/zookeeper/consumer.go
+++ b/registry/zookeeper/consumer.go
@@ -1,96 +1,91 @@
 package zookeeper
 
 import (
-	"fmt"
-)
-
-import (
-	log "github.com/AlexStocks/log4go"
 	jerrors "github.com/juju/errors"
 )
 
 import (
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
 )
 
 // name: service@protocol
-func (r *ZkRegistry) GetService(conf registry.ServiceConfig) ([]registry.ServiceURL, error) {
-
-	var (
-		err         error
-		dubboPath   string
-		nodes       []string
-		listener    *zkEventListener
-		serviceURL  registry.ServiceURL
-		serviceConf registry.ServiceConfig
-		ok          bool
-	)
-	r.listenerLock.Lock()
-	listener = r.listener
-	r.listenerLock.Unlock()
-
-	if listener != nil {
-		listener.listenServiceEvent(conf)
-	}
-
-	r.cltLock.Lock()
-	serviceConf, ok = r.services[conf.Key()]
-	r.cltLock.Unlock()
-	if !ok {
-		return nil, jerrors.Errorf("Service{%s} has not been registered", conf.Key())
-	}
-	if !ok {
-		return nil, jerrors.Errorf("Service{%s}: failed to get serviceConfigIf type", conf.Key())
-	}
-
-	dubboPath = fmt.Sprintf("/dubbo/%s/providers", conf.Service())
-	err = r.validateZookeeperClient()
-	if err != nil {
-		return nil, jerrors.Trace(err)
-	}
-	r.cltLock.Lock()
-	nodes, err = r.client.getChildren(dubboPath)
-	r.cltLock.Unlock()
-	if err != nil {
-		log.Warn("getChildren(dubboPath{%s}) = error{%v}", dubboPath, err)
-		return nil, jerrors.Trace(err)
-	}
-
-	var listenerServiceMap = make(map[string]registry.ServiceURL)
-	for _, n := range nodes {
-
-		serviceURL, err = plugins.DefaultServiceURL()(n)
-		if err != nil {
-			log.Error("NewDefaultServiceURL({%s}) = error{%v}", n, err)
-			continue
-		}
-		if !serviceConf.ServiceEqual(serviceURL) {
-			log.Warn("serviceURL{%s} is not compatible with ServiceConfig{%#v}", serviceURL, serviceConf)
-			continue
-		}
-
-		_, ok := listenerServiceMap[serviceURL.Query().Get(serviceURL.Location())]
-		if !ok {
-			listenerServiceMap[serviceURL.Location()] = serviceURL
-			continue
-		}
-	}
-
-	var services []registry.ServiceURL
-	for _, service := range listenerServiceMap {
-		services = append(services, service)
-	}
-
-	return services, nil
-}
-
-func (r *ZkRegistry) Subscribe() (registry.Listener, error) {
+//func (r *ZkRegistry) GetService(conf registry.ReferenceConfig) ([]config.SubURL, error) {
+//
+//	var (
+//		err         error
+//		dubboPath   string
+//		nodes       []string
+//		listener    *zkEventListener
+//		serviceURL  config.SubURL
+//		serviceConf registry.ReferenceConfig
+//		ok          bool
+//	)
+//	r.listenerLock.Lock()
+//	listener = r.listener
+//	r.listenerLock.Unlock()
+//
+//	if listener != nil {
+//		listener.listenServiceEvent(conf)
+//	}
+//
+//	r.cltLock.Lock()
+//	serviceConf, ok = r.services[conf.Key()]
+//	r.cltLock.Unlock()
+//	if !ok {
+//		return nil, jerrors.Errorf("Path{%s} has not been registered", conf.Key())
+//	}
+//	if !ok {
+//		return nil, jerrors.Errorf("Path{%s}: failed to get serviceConfigIf type", conf.Key())
+//	}
+//
+//	dubboPath = fmt.Sprintf("/dubbo/%s/providers", conf.Path())
+//	err = r.validateZookeeperClient()
+//	if err != nil {
+//		return nil, jerrors.Trace(err)
+//	}
+//	r.cltLock.Lock()
+//	nodes, err = r.client.getChildren(dubboPath)
+//	r.cltLock.Unlock()
+//	if err != nil {
+//		log.Warn("getChildren(dubboPath{%s}) = error{%v}", dubboPath, err)
+//		return nil, jerrors.Trace(err)
+//	}
+//
+//	var listenerServiceMap = make(map[string]config.SubURL)
+//	for _, n := range nodes {
+//
+//		serviceURL, err = plugins.DefaultServiceURL()(n)
+//		if err != nil {
+//			log.Error("NewURL({%s}) = error{%v}", n, err)
+//			continue
+//		}
+//		if !serviceConf.ServiceEqual(serviceURL) {
+//			log.Warn("serviceURL{%s} is not compatible with ReferenceConfig{%#v}", serviceURL, serviceConf)
+//			continue
+//		}
+//
+//		_, ok := listenerServiceMap[serviceURL.Params().Get(serviceURL.Location())]
+//		if !ok {
+//			listenerServiceMap[serviceURL.Location()] = serviceURL
+//			continue
+//		}
+//	}
+//
+//	var services []config.SubURL
+//	for _, service := range listenerServiceMap {
+//		services = append(services, service)
+//	}
+//
+//	return services, nil
+//}
+
+func (r *ZkRegistry) Subscribe(conf config.URL) (registry.Listener, error) {
 	r.wg.Add(1)
-	return r.getListener()
+	return r.getListener(conf)
 }
 
-func (r *ZkRegistry) getListener() (*zkEventListener, error) {
+func (r *ZkRegistry) getListener(conf config.URL) (*zkEventListener, error) {
 	var (
 		zkListener *zkEventListener
 	)
@@ -119,7 +114,9 @@ func (r *ZkRegistry) getListener() (*zkEventListener, error) {
 	// listen
 	r.cltLock.Lock()
 	for _, svs := range r.services {
-		go zkListener.listenServiceEvent(svs)
+		if svs.URLEqual(conf) {
+			go zkListener.listenServiceEvent(svs)
+		}
 	}
 	r.cltLock.Unlock()
 
diff --git a/registry/zookeeper/listener.go b/registry/zookeeper/listener.go
index 1c182cae2b1295c98d6d0c1583a0dbdfba1c50d3..54e48b901f70fb9a5d08ebeabbaff98a2d4800e2 100644
--- a/registry/zookeeper/listener.go
+++ b/registry/zookeeper/listener.go
@@ -1,6 +1,7 @@
 package zookeeper
 
 import (
+	"context"
 	"fmt"
 	"path"
 	"sync"
@@ -14,8 +15,8 @@ import (
 )
 
 import (
-	"github.com/dubbo/dubbo-go/plugins"
-	"github.com/dubbo/dubbo-go/registry"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
 )
 
 const (
@@ -83,7 +84,7 @@ func (l *zkEventListener) listenServiceNodeEvent(zkPath string) bool {
 	return false
 }
 
-func (l *zkEventListener) handleZkNodeEvent(zkPath string, children []string, conf registry.ServiceConfig) {
+func (l *zkEventListener) handleZkNodeEvent(zkPath string, children []string, conf config.URL) {
 	contains := func(s []string, e string) bool {
 		for _, a := range s {
 			if a == e {
@@ -103,7 +104,7 @@ func (l *zkEventListener) handleZkNodeEvent(zkPath string, children []string, co
 	// a node was added -- listen the new node
 	var (
 		newNode    string
-		serviceURL registry.ServiceURL
+		serviceURL config.URL
 	)
 	for _, n := range newChildren {
 		if contains(children, n) {
@@ -112,19 +113,20 @@ func (l *zkEventListener) handleZkNodeEvent(zkPath string, children []string, co
 
 		newNode = path.Join(zkPath, n)
 		log.Info("add zkNode{%s}", newNode)
-		serviceURL, err = plugins.DefaultServiceURL()(n)
+		//context.TODO
+		serviceURL, err = config.NewURL(context.TODO(), n)
 		if err != nil {
-			log.Error("NewDefaultServiceURL(%s) = error{%v}", n, jerrors.ErrorStack(err))
+			log.Error("NewURL(%s) = error{%v}", n, jerrors.ErrorStack(err))
 			continue
 		}
-		if !conf.ServiceEqual(serviceURL) {
-			log.Warn("serviceURL{%s} is not compatible with ServiceConfig{%#v}", serviceURL, conf)
+		if !conf.URLEqual(serviceURL) {
+			log.Warn("serviceURL{%s} is not compatible with SubURL{%#v}", serviceURL.Key(), conf.Key())
 			continue
 		}
 		log.Info("add serviceURL{%s}", serviceURL)
 		l.events <- zkEvent{&registry.ServiceEvent{Action: registry.ServiceAdd, Service: serviceURL}, nil}
 		// listen l service node
-		go func(node string, serviceURL registry.ServiceURL) {
+		go func(node string, serviceURL config.URL) {
 			log.Info("delete zkNode{%s}", node)
 			if l.listenServiceNodeEvent(node) {
 				log.Info("delete serviceURL{%s}", serviceURL)
@@ -143,21 +145,21 @@ func (l *zkEventListener) handleZkNodeEvent(zkPath string, children []string, co
 
 		oldNode = path.Join(zkPath, n)
 		log.Warn("delete zkPath{%s}", oldNode)
-		serviceURL, err = registry.NewDefaultServiceURL(n)
-		if !conf.ServiceEqual(serviceURL) {
-			log.Warn("serviceURL{%s} has been deleted is not compatible with ServiceConfig{%#v}", serviceURL, conf)
+		serviceURL, err = config.NewURL(context.TODO(), n)
+		if !conf.URLEqual(serviceURL) {
+			log.Warn("serviceURL{%s} has been deleted is not compatible with SubURL{%#v}", serviceURL.Key(), conf.Key())
 			continue
 		}
 		log.Warn("delete serviceURL{%s}", serviceURL)
 		if err != nil {
-			log.Error("NewDefaultServiceURL(i{%s}) = error{%v}", n, jerrors.ErrorStack(err))
+			log.Error("NewURL(i{%s}) = error{%v}", n, jerrors.ErrorStack(err))
 			continue
 		}
 		l.events <- zkEvent{&registry.ServiceEvent{Action: registry.ServiceDel, Service: serviceURL}, nil}
 	}
 }
 
-func (l *zkEventListener) listenDirEvent(zkPath string, conf registry.ServiceConfig) {
+func (l *zkEventListener) listenDirEvent(zkPath string, conf config.URL) {
 	l.wg.Add(1)
 	defer l.wg.Done()
 
@@ -193,7 +195,7 @@ func (l *zkEventListener) listenDirEvent(zkPath string, conf registry.ServiceCon
 				continue
 			case <-l.client.done():
 				l.client.unregisterEvent(zkPath, &event)
-				log.Warn("client.done(), listen(path{%s}, ServiceConfig{%#v}) goroutine exit now...", zkPath, conf)
+				log.Warn("client.done(), listen(path{%s}, ReferenceConfig{%#v}) goroutine exit now...", zkPath, conf)
 				return
 			case <-event:
 				log.Info("get zk.EventNodeDataChange notify event")
@@ -213,7 +215,7 @@ func (l *zkEventListener) listenDirEvent(zkPath string, conf registry.ServiceCon
 			}
 			l.handleZkNodeEvent(zkEvent.Path, children, conf)
 		case <-l.client.done():
-			log.Warn("client.done(), listen(path{%s}, ServiceConfig{%#v}) goroutine exit now...", zkPath, conf)
+			log.Warn("client.done(), listen(path{%s}, ReferenceConfig{%#v}) goroutine exit now...", zkPath, conf)
 			return
 		}
 	}
@@ -223,16 +225,16 @@ func (l *zkEventListener) listenDirEvent(zkPath string, conf registry.ServiceCon
 // registry.go:Listen -> listenServiceEvent -> listenDirEvent -> listenServiceNodeEvent
 //                            |
 //                            --------> listenServiceNodeEvent
-func (l *zkEventListener) listenServiceEvent(conf registry.ServiceConfig) {
+func (l *zkEventListener) listenServiceEvent(conf config.URL) {
 	var (
 		err        error
 		zkPath     string
 		dubboPath  string
 		children   []string
-		serviceURL registry.ServiceURL
+		serviceURL config.URL
 	)
 
-	zkPath = fmt.Sprintf("/dubbo/%s/providers", conf.Service())
+	zkPath = fmt.Sprintf("/dubbo%s/providers", conf.Path)
 
 	l.serviceMapLock.Lock()
 	_, ok := l.serviceMap[zkPath]
@@ -254,14 +256,13 @@ func (l *zkEventListener) listenServiceEvent(conf registry.ServiceConfig) {
 	}
 
 	for _, c := range children {
-
-		serviceURL, err = plugins.DefaultServiceURL()(c)
+		serviceURL, err = config.NewURL(context.TODO(), c)
 		if err != nil {
-			log.Error("NewDefaultServiceURL(r{%s}) = error{%v}", c, err)
+			log.Error("NewURL(r{%s}) = error{%v}", c, err)
 			continue
 		}
-		if !conf.ServiceEqual(serviceURL) {
-			log.Warn("serviceURL{%s} is not compatible with ServiceConfig{%#v}", serviceURL, conf)
+		if !conf.URLEqual(serviceURL) {
+			log.Warn("serviceURL %v is not compatible with SubURL %v", serviceURL.Key(), conf.Key())
 			continue
 		}
 		log.Debug("add serviceUrl{%s}", serviceURL)
@@ -270,7 +271,7 @@ func (l *zkEventListener) listenServiceEvent(conf registry.ServiceConfig) {
 		// listen l service node
 		dubboPath = path.Join(zkPath, c)
 		log.Info("listen dubbo service key{%s}", dubboPath)
-		go func(zkPath string, serviceURL registry.ServiceURL) {
+		go func(zkPath string, serviceURL config.URL) {
 			if l.listenServiceNodeEvent(dubboPath) {
 				log.Debug("delete serviceUrl{%s}", serviceURL)
 				l.events <- zkEvent{&registry.ServiceEvent{Action: registry.ServiceDel, Service: serviceURL}, nil}
@@ -280,7 +281,7 @@ func (l *zkEventListener) listenServiceEvent(conf registry.ServiceConfig) {
 	}
 
 	log.Info("listen dubbo path{%s}", zkPath)
-	go func(zkPath string, conf registry.ServiceConfig) {
+	go func(zkPath string, conf config.URL) {
 		l.listenDirEvent(zkPath, conf)
 		log.Warn("listenDirEvent(zkPath{%s}) goroutine exit now", zkPath)
 	}(zkPath, conf)
diff --git a/registry/zookeeper/registry.go b/registry/zookeeper/registry.go
index fa01b045792934e1fd9f8ead1f66be81312edbb4..9bdfc961238c1142f305a1e6b294cca1f15f7a38 100644
--- a/registry/zookeeper/registry.go
+++ b/registry/zookeeper/registry.go
@@ -1,11 +1,12 @@
 package zookeeper
 
 import (
+	"context"
 	"fmt"
-	"github.com/dubbo/dubbo-go/plugins"
 	"net/url"
 	"os"
 	"strconv"
+	"strings"
 	"sync"
 	"time"
 )
@@ -18,8 +19,11 @@ import (
 )
 
 import (
-	"github.com/dubbo/dubbo-go/registry"
-	"github.com/dubbo/dubbo-go/version"
+	"github.com/dubbo/go-for-apache-dubbo/common/constant"
+	"github.com/dubbo/go-for-apache-dubbo/common/extension"
+	"github.com/dubbo/go-for-apache-dubbo/config"
+	"github.com/dubbo/go-for-apache-dubbo/registry"
+	"github.com/dubbo/go-for-apache-dubbo/version"
 )
 
 const (
@@ -36,37 +40,8 @@ var (
 func init() {
 	processID = fmt.Sprintf("%d", os.Getpid())
 	localIP, _ = gxnet.GetLocalIP()
-	plugins.PluggableRegistries["zookeeper"] = NewZkRegistry
-}
-
-type ZkRegistryConfig struct {
-	Address    []string      `required:"true" yaml:"address"  json:"address,omitempty"`
-	UserName   string        `yaml:"user_name" json:"user_name,omitempty"`
-	Password   string        `yaml:"password" json:"password,omitempty"`
-	TimeoutStr string        `yaml:"timeout" default:"5s" json:"timeout,omitempty"` // unit: second
-	Timeout    time.Duration `yaml:"-"  json:"-"`
-}
-
-type Options struct {
-	registry.Options
-	ZkRegistryConfig
-}
-
-func (o Options) ToString() string {
-	return fmt.Sprintf("%s, address:%+v, user:%s, password:%s, conn-timeout:%s",
-		o.Options, o.Address, o.UserName, o.Password, o.Timeout)
-}
-
-type Option func(*Options)
-
-func (Option) Name() string {
-	return "dubbogo-zookeeper-registry-option"
-}
-
-func WithRegistryConf(conf ZkRegistryConfig) Option {
-	return func(o *Options) {
-		o.ZkRegistryConfig = conf
-	}
+	//plugins.PluggableRegistries["zookeeper"] = NewZkRegistry
+	extension.SetRegistry("zookeeper", NewZkRegistry)
 }
 
 /////////////////////////////////////
@@ -74,14 +49,15 @@ func WithRegistryConf(conf ZkRegistryConfig) Option {
 /////////////////////////////////////
 
 type ZkRegistry struct {
-	Options
+	context context.Context
+	*config.URL
 	birth int64          // time of file birth, seconds since Epoch; 0 if unknown
 	wg    sync.WaitGroup // wg+done for zk restart
 	done  chan struct{}
 
 	cltLock  sync.Mutex
 	client   *zookeeperClient
-	services map[string]registry.ServiceConfig // service name + protocol -> service config
+	services map[string]config.URL // service name + protocol -> service config
 
 	listenerLock sync.Mutex
 	listener     *zkEventListener
@@ -90,59 +66,47 @@ type ZkRegistry struct {
 	zkPath map[string]int // key = protocol://ip:port/interface
 }
 
-func NewZkRegistry(opts ...registry.RegistryOption) (registry.Registry, error) {
+func NewZkRegistry(url *config.URL) (registry.Registry, error) {
 	var (
 		err error
 		r   *ZkRegistry
 	)
 
 	r = &ZkRegistry{
+		URL:      url,
 		birth:    time.Now().UnixNano(),
 		done:     make(chan struct{}),
-		services: make(map[string]registry.ServiceConfig),
+		services: make(map[string]config.URL),
 		zkPath:   make(map[string]int),
 	}
 
-	for _, opt := range opts {
-		if o, ok := opt.(Option); ok {
-			o(&r.Options)
-		} else if o, ok := opt.(registry.Option); ok {
-			o(&r.Options.Options)
-		} else {
-			return nil, jerrors.New("option is not available")
-		}
-
-	}
-	//if r.DubboType == 0{
-	//	return nil ,errors.New("Dubbo type should be specified.")
+	//if r.SubURL.Name == "" {
+	//	r.SubURL.Name = RegistryZkClient
+	//}
+	//if r.Version == "" {
+	//	r.Version = version.Version
 	//}
-	if r.Name == "" {
-		r.Name = RegistryZkClient
-	}
-	if r.Version == "" {
-		r.Version = version.Version
-	}
 
-	if r.ZkRegistryConfig.Timeout == 0 {
-		r.ZkRegistryConfig.Timeout = 1e9
-	}
 	err = r.validateZookeeperClient()
 	if err != nil {
-		return nil, jerrors.Trace(err)
+		return nil, err
 	}
 
 	r.wg.Add(1)
 	go r.handleZkRestart()
 
-	//if r.DubboType == registry.CONSUMER {
+	//if r.RoleType == registry.CONSUMER {
 	//	r.wg.Add(1)
 	//	go r.listen()
 	//}
 
 	return r, nil
 }
+func (r *ZkRegistry) GetUrl() config.URL {
+	return *r.URL
+}
 
-func (r *ZkRegistry) Close() {
+func (r *ZkRegistry) Destroy() {
 	close(r.done)
 	r.wg.Wait()
 	r.closeRegisters()
@@ -157,23 +121,30 @@ func (r *ZkRegistry) validateZookeeperClient() error {
 	r.cltLock.Lock()
 	defer r.cltLock.Unlock()
 	if r.client == nil {
-		r.client, err = newZookeeperClient(RegistryZkClient, r.Address, r.ZkRegistryConfig.Timeout)
+		//in dubbp ,every registry only connect one node ,so this is []string{r.Address}
+		timeout, err := time.ParseDuration(r.GetParam(constant.REGISTRY_TIMEOUT_KEY, constant.DEFAULT_REG_TIMEOUT))
+		if err != nil {
+			log.Error("timeout config %v is invalid ,err is %v",
+				r.GetParam(constant.REGISTRY_TIMEOUT_KEY, constant.DEFAULT_REG_TIMEOUT), err.Error())
+			return jerrors.Annotatef(err, "newZookeeperClient(address:%+v)", r.Location)
+		}
+		r.client, err = newZookeeperClient(RegistryZkClient, []string{r.Location}, timeout)
 		if err != nil {
 			log.Warn("newZookeeperClient(name{%s}, zk addresss{%v}, timeout{%d}) = error{%v}",
-				RegistryZkClient, r.Address, r.Timeout.String(), err)
-			return jerrors.Annotatef(err, "newZookeeperClient(address:%+v)", r.Address)
+				RegistryZkClient, r.Location, timeout.String(), err)
+			return jerrors.Annotatef(err, "newZookeeperClient(address:%+v)", r.Location)
 		}
 	}
 	if r.client.conn == nil {
 		var event <-chan zk.Event
 		r.client.conn, event, err = zk.Connect(r.client.zkAddrs, r.client.timeout)
-		if err != nil {
+		if err == nil {
 			r.client.wait.Add(1)
 			go r.client.handleZkEvent(event)
 		}
 	}
 
-	return jerrors.Annotatef(err, "newZookeeperClient(address:%+v)", r.Address)
+	return jerrors.Annotatef(err, "newZookeeperClient(address:%+v)", r.PrimitiveURL)
 }
 
 func (r *ZkRegistry) handleZkRestart() {
@@ -181,8 +152,8 @@ func (r *ZkRegistry) handleZkRestart() {
 		err       error
 		flag      bool
 		failTimes int
-		confIf    registry.ServiceConfig
-		services  []registry.ServiceConfig
+		confIf    config.URL
+		services  []config.URL
 	)
 
 	defer r.wg.Done()
@@ -214,6 +185,7 @@ LOOP:
 				if err == nil {
 					// copy r.services
 					r.cltLock.Lock()
+					services = []config.URL{}
 					for _, confIf = range r.services {
 						services = append(services, confIf)
 					}
@@ -242,20 +214,21 @@ LOOP:
 	}
 }
 
-func (r *ZkRegistry) Register(conf registry.ServiceConfig) error {
+func (r *ZkRegistry) Register(conf config.URL) error {
 	var (
 		ok       bool
 		err      error
 		listener *zkEventListener
 	)
-	switch r.DubboType {
-	case registry.CONSUMER:
+	role, _ := strconv.Atoi(r.URL.GetParam(constant.ROLE_KEY, ""))
+	switch role {
+	case config.CONSUMER:
 		ok = false
 		r.cltLock.Lock()
 		_, ok = r.services[conf.Key()]
 		r.cltLock.Unlock()
 		if ok {
-			return jerrors.Errorf("Service{%s} has been registered", conf.Service())
+			return jerrors.Errorf("Path{%s} has been registered", conf.Path)
 		}
 
 		err = r.register(conf)
@@ -274,17 +247,17 @@ func (r *ZkRegistry) Register(conf registry.ServiceConfig) error {
 		if listener != nil {
 			go listener.listenServiceEvent(conf)
 		}
-	case registry.PROVIDER:
+	case config.PROVIDER:
 
 		// 检验服务是否已经注册过
 		ok = false
 		r.cltLock.Lock()
-		// 注意此处与consumerZookeeperRegistry的差异,consumer用的是conf.Service,
+		// 注意此处与consumerZookeeperRegistry的差异,consumer用的是conf.Path,
 		// 因为consumer要提供watch功能给selector使用, provider允许注册同一个service的多个group or version
-		_, ok = r.services[conf.String()]
+		_, ok = r.services[conf.Key()]
 		r.cltLock.Unlock()
 		if ok {
-			return jerrors.Errorf("Service{%s} has been registered", conf.String())
+			return jerrors.Errorf("Path{%s} has been registered", conf.Key())
 		}
 
 		err = r.register(conf)
@@ -293,7 +266,7 @@ func (r *ZkRegistry) Register(conf registry.ServiceConfig) error {
 		}
 
 		r.cltLock.Lock()
-		r.services[conf.String()] = conf
+		r.services[conf.Key()] = conf
 		r.cltLock.Unlock()
 
 		log.Debug("(ZkProviderRegistry)Register(conf{%#v})", conf)
@@ -302,17 +275,16 @@ func (r *ZkRegistry) Register(conf registry.ServiceConfig) error {
 	return nil
 }
 
-func (r *ZkRegistry) register(c registry.ServiceConfig) error {
+func (r *ZkRegistry) register(c config.URL) error {
 	var (
-		err        error
-		revision   string
+		err error
+		//revision   string
 		params     url.Values
 		urlPath    string
 		rawURL     string
 		encodedURL string
 		dubboPath  string
-		conf       registry.ProviderServiceConfig
-		ok         bool
+		//conf       config.URL
 	)
 
 	err = r.validateZookeeperClient()
@@ -320,35 +292,24 @@ func (r *ZkRegistry) register(c registry.ServiceConfig) error {
 		return jerrors.Trace(err)
 	}
 	params = url.Values{}
+	for k, v := range c.Params {
+		params[k] = v
+	}
 
-	params.Add("application", r.ApplicationConfig.Name)
-	params.Add("default.timeout", fmt.Sprintf("%d", defaultTimeout/1e6))
-	params.Add("environment", r.ApplicationConfig.Environment)
-	params.Add("org", r.ApplicationConfig.Organization)
-	params.Add("module", r.ApplicationConfig.Module)
-	params.Add("owner", r.ApplicationConfig.Owner)
 	params.Add("pid", processID)
 	params.Add("ip", localIP)
-	params.Add("timeout", fmt.Sprintf("%d", int64(r.Timeout)/1e6))
-	params.Add("timestamp", fmt.Sprintf("%d", r.birth/1e6))
+	//params.Add("timeout", fmt.Sprintf("%d", int64(r.Timeout)/1e6))
 
-	revision = r.ApplicationConfig.Version
-	if revision == "" {
-		revision = "0.1.0"
-	}
-	params.Add("revision", revision) // revision是pox.xml中application的version属性的值
+	role, _ := strconv.Atoi(r.URL.GetParam(constant.ROLE_KEY, ""))
+	switch role {
 
-	switch r.DubboType {
+	case config.PROVIDER:
 
-	case registry.PROVIDER:
-		if conf, ok = c.(registry.ProviderServiceConfig); !ok {
-			return jerrors.Errorf("conf is not ProviderServiceConfig")
-		}
-		if conf.Service() == "" || conf.Methods() == "" {
-			return jerrors.Errorf("conf{Service:%s, Methods:%s}", conf.Service(), conf.Methods())
+		if c.Path == "" || len(c.Methods) == 0 {
+			return jerrors.Errorf("conf{Path:%s, Methods:%s}", c.Path, c.Methods)
 		}
 		// 先创建服务下面的provider node
-		dubboPath = fmt.Sprintf("/dubbo/%s/%s", conf.Service(), registry.DubboNodes[registry.PROVIDER])
+		dubboPath = fmt.Sprintf("/dubbo%s/%s", c.Path, config.DubboNodes[config.PROVIDER])
 		r.cltLock.Lock()
 		err = r.client.Create(dubboPath)
 		r.cltLock.Unlock()
@@ -357,45 +318,40 @@ func (r *ZkRegistry) register(c registry.ServiceConfig) error {
 			return jerrors.Annotatef(err, "zkclient.Create(path:%s)", dubboPath)
 		}
 		params.Add("anyhost", "true")
-		params.Add("interface", conf.Service())
 
-		if conf.Group() != "" {
-			params.Add("group", conf.Group())
-		}
 		// dubbo java consumer来启动找provider url时,因为category不匹配,会找不到provider,导致consumer启动不了,所以使用consumers&providers
 		// DubboRole               = [...]string{"consumer", "", "", "provider"}
-		// params.Add("category", (DubboType(PROVIDER)).Role())
-		params.Add("category", (registry.DubboType(registry.PROVIDER)).String())
+		// params.Add("category", (RoleType(PROVIDER)).Role())
+		params.Add("category", (config.RoleType(config.PROVIDER)).String())
 		params.Add("dubbo", "dubbo-provider-golang-"+version.Version)
 
-		params.Add("side", (registry.DubboType(registry.PROVIDER)).Role())
+		params.Add("side", (config.RoleType(config.PROVIDER)).Role())
 
-		if conf.Version() != "" {
-			params.Add("version", conf.Version())
-		}
-		if conf.Methods() != "" {
-			params.Add("methods", conf.Methods())
+		if len(c.Methods) == 0 {
+			params.Add("methods", strings.Join(c.Methods, ","))
 		}
 		log.Debug("provider zk url params:%#v", params)
-		var path = conf.Path()
-		if path == "" {
-			path = localIP
+		var host string
+		if c.Ip == "" {
+			host = localIP + ":" + c.Port
+		} else {
+			host = c.Ip + ":" + c.Port
 		}
 
-		urlPath = conf.Service()
+		urlPath = c.Path
 		if r.zkPath[urlPath] != 0 {
 			urlPath += strconv.Itoa(r.zkPath[urlPath])
 		}
 		r.zkPath[urlPath]++
-		rawURL = fmt.Sprintf("%s://%s/%s?%s", conf.Protocol(), path, urlPath, params.Encode())
+		rawURL = fmt.Sprintf("%s://%s%s?%s", c.Protocol, host, urlPath, params.Encode())
 		encodedURL = url.QueryEscape(rawURL)
 
 		// 把自己注册service providers
-		dubboPath = fmt.Sprintf("/dubbo/%s/%s", conf.Service(), (registry.DubboType(registry.PROVIDER)).String())
+		dubboPath = fmt.Sprintf("/dubbo%s/%s", c.Path, (config.RoleType(config.PROVIDER)).String())
 		log.Debug("provider path:%s, url:%s", dubboPath, rawURL)
 
-	case registry.CONSUMER:
-		dubboPath = fmt.Sprintf("/dubbo/%s/%s", c.Service(), registry.DubboNodes[registry.CONSUMER])
+	case config.CONSUMER:
+		dubboPath = fmt.Sprintf("/dubbo%s/%s", c.Path, config.DubboNodes[config.CONSUMER])
 		r.cltLock.Lock()
 		err = r.client.Create(dubboPath)
 		r.cltLock.Unlock()
@@ -403,7 +359,7 @@ func (r *ZkRegistry) register(c registry.ServiceConfig) error {
 			log.Error("zkClient.create(path{%s}) = error{%v}", dubboPath, jerrors.ErrorStack(err))
 			return jerrors.Trace(err)
 		}
-		dubboPath = fmt.Sprintf("/dubbo/%s/%s", c.Service(), registry.DubboNodes[registry.PROVIDER])
+		dubboPath = fmt.Sprintf("/dubbo%s/%s", c.Path, config.DubboNodes[config.PROVIDER])
 		r.cltLock.Lock()
 		err = r.client.Create(dubboPath)
 		r.cltLock.Unlock()
@@ -412,29 +368,18 @@ func (r *ZkRegistry) register(c registry.ServiceConfig) error {
 			return jerrors.Trace(err)
 		}
 
-		params.Add("protocol", c.Protocol())
-		params.Add("interface", c.Service())
-		revision = r.ApplicationConfig.Version
-		if revision == "" {
-			revision = "0.1.0"
-		}
-		params.Add("revision", revision)
-		if c.Group() != "" {
-			params.Add("group", c.Group())
-		}
-		params.Add("category", (registry.DubboType(registry.CONSUMER)).String())
+		params.Add("protocol", c.Protocol)
+
+		params.Add("category", (config.RoleType(config.CONSUMER)).String())
 		params.Add("dubbo", "dubbogo-consumer-"+version.Version)
 
-		if c.Version() != "" {
-			params.Add("version", c.Version())
-		}
-		rawURL = fmt.Sprintf("consumer://%s/%s?%s", localIP, c.Service()+c.Version(), params.Encode())
+		rawURL = fmt.Sprintf("consumer://%s%s?%s", localIP, c.Path, params.Encode())
 		encodedURL = url.QueryEscape(rawURL)
 
-		dubboPath = fmt.Sprintf("/dubbo/%s/%s", c.Service(), (registry.DubboType(registry.CONSUMER)).String())
+		dubboPath = fmt.Sprintf("/dubbo%s/%s", c.Path, (config.RoleType(config.CONSUMER)).String())
 		log.Debug("consumer path:%s, url:%s", dubboPath, rawURL)
 	default:
-		return jerrors.Errorf("@c{%v} type is not DefaultServiceConfig or DefaultProviderServiceConfig", c)
+		return jerrors.Errorf("@c{%v} type is not referencer or provider", c)
 	}
 
 	err = r.registerTempZookeeperNode(dubboPath, encodedURL)
@@ -478,11 +423,11 @@ func (r *ZkRegistry) closeRegisters() {
 	r.services = nil
 }
 
-func (r *ZkRegistry) IsClosed() bool {
+func (r *ZkRegistry) IsAvailable() bool {
 	select {
 	case <-r.done:
-		return true
-	default:
 		return false
+	default:
+		return true
 	}
 }
diff --git a/registry/zookeeper/zk_client.go b/registry/zookeeper/zk_client.go
index a33a5890d513e0174c9ec527f044d890e1402a91..729163755d6852fd73e130f8885bb0542a497730 100644
--- a/registry/zookeeper/zk_client.go
+++ b/registry/zookeeper/zk_client.go
@@ -44,7 +44,7 @@ func stateToString(state zk.State) string {
 	case zk.StateExpired:
 		return "zookeeper connection expired"
 	case zk.StateConnected:
-		return "zookeeper conneced"
+		return "zookeeper connected"
 	case zk.StateHasSession:
 		return "zookeeper has session"
 	case zk.StateUnknown:
@@ -134,7 +134,7 @@ LOOP:
 				}
 				z.Unlock()
 			case (int)(zk.StateConnecting), (int)(zk.StateConnected), (int)(zk.StateHasSession):
-				if state != (int)(zk.StateConnecting) || state != (int)(zk.StateDisconnected) {
+				if state == (int)(zk.StateHasSession) {
 					continue
 				}
 				if a, ok := z.eventRegistry[event.Path]; ok && 0 < len(a) {
diff --git a/server/config.go b/server/config.go
deleted file mode 100644
index cf6eb1788b754ff6c493e69d9177546940294a3d..0000000000000000000000000000000000000000
--- a/server/config.go
+++ /dev/null
@@ -1,13 +0,0 @@
-package server
-
-import "github.com/AlexStocks/goext/net"
-
-type ServerConfig struct {
-	Protocol string `required:"true",default:"dubbo" yaml:"protocol" json:"protocol,omitempty"` // codec string, jsonrpc  etc
-	IP       string `yaml:"ip" json:"ip,omitempty"`
-	Port     int    `required:"true" yaml:"port" json:"port,omitempty"`
-}
-
-func (c *ServerConfig) Address() string {
-	return gxnet.HostAddress(c.IP, c.Port)
-}
diff --git a/version/version.go b/version/version.go
index 7c36ca6a75aec24dc2932b940536fbffe3a16364..d5b7850cbd05400ae04d73febe62889068b6e617 100644
--- a/version/version.go
+++ b/version/version.go
@@ -1,7 +1,7 @@
 package version
 
 const (
-	Version = "1.0.0"
+	Version = "2.6.0"
 	Name    = "dubbogo"
-	DATE    = "2019/04/17"
+	DATE    = "2019/05/06"
 )