Merge branch 'main' into feature-triple

# Conflicts:
#	go.mod
#	metrics/prometheus/reporter.go
#	metrics/rpc/util.go
diff --git a/client/action.go b/client/action.go
new file mode 100644
index 0000000..58893e5
--- /dev/null
+++ b/client/action.go
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package client
+
+import (
+	"fmt"
+	"net/url"
+	"os"
+	"strconv"
+	"time"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+	gxstrings "github.com/dubbogo/gost/strings"
+
+	constant2 "github.com/dubbogo/triple/pkg/common/constant"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/cluster/directory/static"
+	"dubbo.apache.org/dubbo-go/v3/common"
+	commonCfg "dubbo.apache.org/dubbo-go/v3/common/config"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/graceful_shutdown"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/protocolwrapper"
+	"dubbo.apache.org/dubbo-go/v3/proxy"
+)
+
+func getEnv(key, fallback string) string {
+	if value, ok := os.LookupEnv(key); ok {
+		return value
+	}
+	return fallback
+}
+
+func updateOrCreateMeshURL(opts *ClientOptions) {
+	ref := opts.Reference
+	con := opts.Consumer
+
+	if ref.URL != "" {
+		logger.Infof("URL specified explicitly %v", ref.URL)
+	}
+
+	if !con.MeshEnabled {
+		return
+	}
+	if ref.Protocol != constant2.TRIPLE {
+		panic(fmt.Sprintf("Mesh mode enabled, Triple protocol expected but %v protocol found!", ref.Protocol))
+	}
+	if ref.ProvidedBy == "" {
+		panic("Mesh mode enabled, provided-by should not be empty!")
+	}
+
+	podNamespace := getEnv(constant.PodNamespaceEnvKey, constant.DefaultNamespace)
+	clusterDomain := getEnv(constant.ClusterDomainKey, constant.DefaultClusterDomain)
+
+	var meshPort int
+	if ref.MeshProviderPort > 0 {
+		meshPort = ref.MeshProviderPort
+	} else {
+		meshPort = constant.DefaultMeshPort
+	}
+
+	ref.URL = "tri://" + ref.ProvidedBy + "." + podNamespace + constant.SVC + clusterDomain + ":" + strconv.Itoa(meshPort)
+}
+
+// ReferWithService retrieves invokers from urls.
+func (opts *ClientOptions) ReferWithService(srv common.RPCService) {
+	opts.refer(srv, nil)
+}
+
+func (opts *ClientOptions) ReferWithInfo(info *ClientInfo) {
+	opts.refer(nil, info)
+}
+
+func (opts *ClientOptions) ReferWithServiceAndInfo(srv common.RPCService, info *ClientInfo) {
+	opts.refer(srv, info)
+}
+
+func (opts *ClientOptions) refer(srv common.RPCService, info *ClientInfo) {
+	ref := opts.Reference
+	con := opts.Consumer
+
+	var methods []string
+	if info != nil {
+		ref.InterfaceName = info.InterfaceName
+		methods = info.MethodNames
+		opts.id = info.InterfaceName
+		opts.info = info
+	} else {
+		opts.id = common.GetReference(srv)
+	}
+	// If adaptive service is enabled,
+	// the cluster and load balance should be overridden to "adaptivesvc" and "p2c" respectively.
+	if con != nil && con.AdaptiveService {
+		ref.Cluster = constant.ClusterKeyAdaptiveService
+		ref.Loadbalance = constant.LoadBalanceKeyP2C
+	}
+
+	// cfgURL is an interface-level invoker url, in the other words, it represents an interface.
+	cfgURL := common.NewURLWithOptions(
+		common.WithPath(ref.InterfaceName),
+		common.WithProtocol(ref.Protocol),
+		common.WithMethods(methods),
+		common.WithParams(opts.getURLMap()),
+		common.WithParamsValue(constant.BeanNameKey, opts.id),
+		common.WithParamsValue(constant.MetadataTypeKey, opts.metaDataType),
+	)
+
+	if ref.ForceTag {
+		cfgURL.AddParam(constant.ForceUseTag, "true")
+	}
+	opts.postProcessConfig(cfgURL)
+
+	// if mesh-enabled is set
+	updateOrCreateMeshURL(opts)
+
+	// retrieving urls from config, and appending the urls to opts.urls
+	if ref.URL != "" { // use user-specific urls
+		/*
+			 Two types of URL are allowed for opts.URL:
+				1. direct url: server IP, that is, no need for a registry anymore
+				2. registry url
+			 They will be handled in different ways:
+			 For example, we have a direct url and a registry url:
+				1. "tri://localhost:10000" is a direct url
+				2. "registry://localhost:2181" is a registry url.
+			 Then, opts.URL looks like a string separated by semicolon: "tri://localhost:10000;registry://localhost:2181".
+			 The result of urlStrings is a string array: []string{"tri://localhost:10000", "registry://localhost:2181"}.
+		*/
+		urlStrings := gxstrings.RegSplit(ref.URL, "\\s*[;]+\\s*")
+		for _, urlStr := range urlStrings {
+			serviceURL, err := common.NewURL(urlStr)
+			if err != nil {
+				panic(fmt.Sprintf("url configuration error,  please check your configuration, user specified URL %v refer error, error message is %v ", urlStr, err.Error()))
+			}
+			if serviceURL.Protocol == constant.RegistryProtocol { // serviceURL in this branch is a registry protocol
+				serviceURL.SubURL = cfgURL
+				opts.urls = append(opts.urls, serviceURL)
+			} else { // serviceURL in this branch is the target endpoint IP address
+				if serviceURL.Path == "" {
+					serviceURL.Path = "/" + ref.InterfaceName
+				}
+				// replace params of serviceURL with params of cfgUrl
+				// other stuff, e.g. IP, port, etc., are same as serviceURL
+				newURL := common.MergeURL(serviceURL, cfgURL)
+				newURL.AddParam("peer", "true")
+				opts.urls = append(opts.urls, newURL)
+			}
+		}
+	} else { // use registry configs
+		opts.urls = config.LoadRegistries(ref.RegistryIDs, opts.registriesCompat, common.CONSUMER)
+		// set url to regURLs
+		for _, regURL := range opts.urls {
+			regURL.SubURL = cfgURL
+		}
+	}
+
+	// Get invokers according to opts.urls
+	var (
+		invoker protocol.Invoker
+		regURL  *common.URL
+	)
+	invokers := make([]protocol.Invoker, len(opts.urls))
+	for i, u := range opts.urls {
+		if u.Protocol == constant.ServiceRegistryProtocol {
+			invoker = extension.GetProtocol(constant.RegistryProtocol).Refer(u)
+		} else {
+			invoker = extension.GetProtocol(u.Protocol).Refer(u)
+		}
+
+		if ref.URL != "" {
+			invoker = protocolwrapper.BuildInvokerChain(invoker, constant.ReferenceFilterKey)
+		}
+
+		invokers[i] = invoker
+		if u.Protocol == constant.RegistryProtocol {
+			regURL = u
+		}
+	}
+
+	// TODO(hxmhlt): decouple from directory, config should not depend on directory module
+	if len(invokers) == 1 {
+		opts.invoker = invokers[0]
+		if ref.URL != "" {
+			hitClu := constant.ClusterKeyFailover
+			if u := opts.invoker.GetURL(); u != nil {
+				hitClu = u.GetParam(constant.ClusterKey, constant.ClusterKeyZoneAware)
+			}
+			cluster, err := extension.GetCluster(hitClu)
+			if err != nil {
+				panic(err)
+			} else {
+				opts.invoker = cluster.Join(static.NewDirectory(invokers))
+			}
+		}
+	} else {
+		var hitClu string
+		if regURL != nil {
+			// for multi-subscription scenario, use 'zone-aware' policy by default
+			hitClu = constant.ClusterKeyZoneAware
+		} else {
+			// not a registry url, must be direct invoke.
+			hitClu = constant.ClusterKeyFailover
+			if u := invokers[0].GetURL(); u != nil {
+				hitClu = u.GetParam(constant.ClusterKey, constant.ClusterKeyZoneAware)
+			}
+		}
+		cluster, err := extension.GetCluster(hitClu)
+		if err != nil {
+			panic(err)
+		} else {
+			opts.invoker = cluster.Join(static.NewDirectory(invokers))
+		}
+	}
+
+	// publish consumer's metadata
+	publishServiceDefinition(cfgURL)
+	// create proxy
+	if info == nil {
+		// todo(DMwangnima): think about a more ideal way
+		if con == nil {
+			panic("client must be configured with ConsumerConfig when using config.Load")
+		}
+
+		if ref.Async {
+			var callback common.CallbackResponse
+			if asyncSrv, ok := srv.(common.AsyncCallbackService); ok {
+				callback = asyncSrv.CallBack
+			}
+			opts.pxy = extension.GetProxyFactory(con.ProxyFactory).GetAsyncProxy(opts.invoker, callback, cfgURL)
+		} else {
+			opts.pxy = extension.GetProxyFactory(con.ProxyFactory).GetProxy(opts.invoker, cfgURL)
+		}
+		opts.pxy.Implement(srv)
+	}
+	// this protocol would be destroyed in graceful_shutdown
+	// please refer to (https://github.com/apache/dubbo-go/issues/2429)
+	graceful_shutdown.RegisterProtocol(ref.Protocol)
+}
+
+func (opts *ClientOptions) CheckAvailable() bool {
+	ref := opts.Reference
+	if opts.invoker == nil {
+		logger.Warnf("The interface %s invoker not exist, may you should check your interface config.", ref.InterfaceName)
+		return false
+	}
+	if !opts.invoker.IsAvailable() {
+		return false
+	}
+	return true
+}
+
+// Implement
+// @v is service provider implemented RPCService
+func (opts *ClientOptions) Implement(v common.RPCService) {
+	if opts.pxy != nil {
+		opts.pxy.Implement(v)
+	} else if opts.info != nil {
+		opts.info.ClientInjectFunc(v, &Client{
+			invoker: opts.invoker,
+			info:    opts.info,
+		})
+	}
+}
+
+// GetRPCService gets RPCService from proxy
+func (opts *ClientOptions) GetRPCService() common.RPCService {
+	return opts.pxy.Get()
+}
+
+// GetProxy gets proxy
+func (opts *ClientOptions) GetProxy() *proxy.Proxy {
+	return opts.pxy
+}
+
+func (opts *ClientOptions) getURLMap() url.Values {
+	ref := opts.Reference
+	app := opts.applicationCompat
+
+	urlMap := url.Values{}
+	// first set user params
+	for k, v := range ref.Params {
+		urlMap.Set(k, v)
+	}
+	urlMap.Set(constant.InterfaceKey, ref.InterfaceName)
+	urlMap.Set(constant.TimestampKey, strconv.FormatInt(time.Now().Unix(), 10))
+	urlMap.Set(constant.ClusterKey, ref.Cluster)
+	urlMap.Set(constant.LoadbalanceKey, ref.Loadbalance)
+	urlMap.Set(constant.RetriesKey, ref.Retries)
+	urlMap.Set(constant.GroupKey, ref.Group)
+	urlMap.Set(constant.VersionKey, ref.Version)
+	urlMap.Set(constant.GenericKey, ref.Generic)
+	urlMap.Set(constant.RegistryRoleKey, strconv.Itoa(common.CONSUMER))
+	urlMap.Set(constant.ProvidedBy, ref.ProvidedBy)
+	urlMap.Set(constant.SerializationKey, ref.Serialization)
+	urlMap.Set(constant.TracingConfigKey, ref.TracingKey)
+
+	urlMap.Set(constant.ReleaseKey, "dubbo-golang-"+constant.Version)
+	urlMap.Set(constant.SideKey, (common.RoleType(common.CONSUMER)).Role())
+
+	if len(ref.RequestTimeout) != 0 {
+		urlMap.Set(constant.TimeoutKey, ref.RequestTimeout)
+	}
+	// getty invoke async or sync
+	urlMap.Set(constant.AsyncKey, strconv.FormatBool(ref.Async))
+	urlMap.Set(constant.StickyKey, strconv.FormatBool(ref.Sticky))
+
+	// applicationConfig info
+	if app != nil {
+		urlMap.Set(constant.ApplicationKey, app.Name)
+		urlMap.Set(constant.OrganizationKey, app.Organization)
+		urlMap.Set(constant.NameKey, app.Name)
+		urlMap.Set(constant.ModuleKey, app.Module)
+		urlMap.Set(constant.AppVersionKey, app.Version)
+		urlMap.Set(constant.OwnerKey, app.Owner)
+		urlMap.Set(constant.EnvironmentKey, app.Environment)
+	}
+
+	// filter
+	defaultReferenceFilter := constant.DefaultReferenceFilters
+	if ref.Generic != "" {
+		defaultReferenceFilter = constant.GenericFilterKey + "," + defaultReferenceFilter
+	}
+	urlMap.Set(constant.ReferenceFilterKey, commonCfg.MergeValue(ref.Filter, "", defaultReferenceFilter))
+
+	for _, v := range ref.Methods {
+		urlMap.Set("methods."+v.Name+"."+constant.LoadbalanceKey, v.LoadBalance)
+		urlMap.Set("methods."+v.Name+"."+constant.RetriesKey, v.Retries)
+		urlMap.Set("methods."+v.Name+"."+constant.StickyKey, strconv.FormatBool(v.Sticky))
+		if len(v.RequestTimeout) != 0 {
+			urlMap.Set("methods."+v.Name+"."+constant.TimeoutKey, v.RequestTimeout)
+		}
+	}
+
+	return urlMap
+}
+
+// todo: figure this out
+//// GenericLoad ...
+//func (opts *ClientOptions) GenericLoad(id string) {
+//	genericService := generic.NewGenericService(opts.id)
+//	config.SetConsumerService(genericService)
+//	opts.id = id
+//	opts.Refer(genericService)
+//	opts.Implement(genericService)
+//}
+
+// GetInvoker get invoker from ReferenceConfigs
+func (opts *ClientOptions) GetInvoker() protocol.Invoker {
+	return opts.invoker
+}
+
+// postProcessConfig asks registered ConfigPostProcessor to post-process the current ReferenceConfigs.
+func (opts *ClientOptions) postProcessConfig(url *common.URL) {
+	for _, p := range extension.GetConfigPostProcessors() {
+		p.PostProcessReferenceConfig(url)
+	}
+}
+
+func publishServiceDefinition(url *common.URL) {
+	localService, err := extension.GetLocalMetadataService(constant.DefaultKey)
+	if err != nil {
+		logger.Warnf("get local metadata service failed, please check if you have imported _ \"dubbo.apache.org/dubbo-go/v3/metadata/service/local\"")
+		return
+	}
+	localService.PublishServiceDefinition(url)
+	if url.GetParam(constant.MetadataTypeKey, "") != constant.RemoteMetadataStorageType {
+		return
+	}
+	if remoteMetadataService, err := extension.GetRemoteMetadataService(); err == nil && remoteMetadataService != nil {
+		remoteMetadataService.PublishServiceDefinition(url)
+	}
+}
diff --git a/client/client.go b/client/client.go
new file mode 100644
index 0000000..8e46a02
--- /dev/null
+++ b/client/client.go
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package client
+
+import (
+	"context"
+)
+
+import (
+	"github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	invocation_impl "dubbo.apache.org/dubbo-go/v3/protocol/invocation"
+)
+
+type Client struct {
+	invoker protocol.Invoker
+	info    *ClientInfo
+
+	cliOpts *ClientOptions
+}
+
+type ClientInfo struct {
+	InterfaceName    string
+	MethodNames      []string
+	ClientInjectFunc func(dubboCliRaw interface{}, cli *Client)
+	Meta             map[string]interface{}
+}
+
+func (cli *Client) call(ctx context.Context, paramsRawVals []interface{}, interfaceName, methodName, callType string, opts ...CallOption) (protocol.Result, error) {
+	// get a default CallOptions
+	// apply CallOption
+	options := newDefaultCallOptions()
+	for _, opt := range opts {
+		opt(options)
+	}
+
+	inv, err := generateInvocation(methodName, paramsRawVals, callType, options)
+	if err != nil {
+		return nil, err
+	}
+	// todo: move timeout into context or invocation
+	return cli.invoker.Invoke(ctx, inv), nil
+
+}
+
+func (cli *Client) CallUnary(ctx context.Context, req, resp interface{}, interfaceName, methodName string, opts ...CallOption) error {
+	res, err := cli.call(ctx, []interface{}{req, resp}, interfaceName, methodName, constant.CallUnary, opts...)
+	if err != nil {
+		return err
+	}
+	return res.Error()
+}
+
+func (cli *Client) CallClientStream(ctx context.Context, interfaceName, methodName string, opts ...CallOption) (interface{}, error) {
+	res, err := cli.call(ctx, nil, interfaceName, methodName, constant.CallClientStream, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return res.Result(), res.Error()
+}
+
+func (cli *Client) CallServerStream(ctx context.Context, req interface{}, interfaceName, methodName string, opts ...CallOption) (interface{}, error) {
+	res, err := cli.call(ctx, []interface{}{req}, interfaceName, methodName, constant.CallServerStream, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return res.Result(), res.Error()
+}
+
+func (cli *Client) CallBidiStream(ctx context.Context, interfaceName, methodName string, opts ...CallOption) (interface{}, error) {
+	res, err := cli.call(ctx, nil, interfaceName, methodName, constant.CallBidiStream, opts...)
+	if err != nil {
+		return nil, err
+	}
+	return res.Result(), res.Error()
+}
+
+func (cli *Client) Init(info *ClientInfo) error {
+	if info == nil {
+		return errors.New("ClientInfo is nil")
+	}
+
+	cli.cliOpts.ReferWithInfo(info)
+	cli.invoker = cli.cliOpts.invoker
+
+	return nil
+}
+
+func generateInvocation(methodName string, paramsRawVals []interface{}, callType string, opts *CallOptions) (protocol.Invocation, error) {
+	inv := invocation_impl.NewRPCInvocationWithOptions(
+		invocation_impl.WithMethodName(methodName),
+		// todo: process opts
+		invocation_impl.WithParameterRawValues(paramsRawVals),
+	)
+	inv.SetAttribute(constant.CallTypeKey, callType)
+
+	return inv, nil
+}
+
+func NewClient(opts ...ClientOption) (*Client, error) {
+	newCliOpts := defaultClientOptions()
+	if err := newCliOpts.init(opts...); err != nil {
+		return nil, err
+	}
+	return &Client{
+		cliOpts: newCliOpts,
+	}, nil
+}
diff --git a/client/compat.go b/client/compat.go
new file mode 100644
index 0000000..b1f8d38
--- /dev/null
+++ b/client/compat.go
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package client
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+// these functions are used to resolve circular dependencies temporarily.
+// please refer to issue(https://github.com/apache/dubbo-go/issues/2377)
+// todo(DMwangnima): remove these functions when refactoring dubbo-go
+func compatApplicationConfig(c *global.ApplicationConfig) *config.ApplicationConfig {
+	return &config.ApplicationConfig{
+		Organization: c.Organization,
+		Name:         c.Name,
+		Module:       c.Module,
+		Group:        c.Group,
+		Version:      c.Version,
+		Owner:        c.Owner,
+		Environment:  c.Environment,
+		MetadataType: c.MetadataType,
+		Tag:          c.Tag,
+	}
+}
+
+func compatRegistryConfig(c *global.RegistryConfig) *config.RegistryConfig {
+	return &config.RegistryConfig{
+		Protocol:          c.Protocol,
+		Timeout:           c.Timeout,
+		Group:             c.Group,
+		Namespace:         c.Namespace,
+		TTL:               c.TTL,
+		Address:           c.Address,
+		Username:          c.Username,
+		Password:          c.Password,
+		Simplified:        c.Simplified,
+		Preferred:         c.Preferred,
+		Zone:              c.Zone,
+		Weight:            c.Weight,
+		Params:            c.Params,
+		RegistryType:      c.RegistryType,
+		UseAsMetaReport:   c.UseAsMetaReport,
+		UseAsConfigCenter: c.UseAsConfigCenter,
+	}
+}
+
+func compatMethodConfig(c *global.MethodConfig) *config.MethodConfig {
+	return &config.MethodConfig{
+		InterfaceId:                 c.InterfaceId,
+		InterfaceName:               c.InterfaceName,
+		Name:                        c.Name,
+		Retries:                     c.Retries,
+		LoadBalance:                 c.LoadBalance,
+		Weight:                      c.Weight,
+		TpsLimitInterval:            c.TpsLimitInterval,
+		TpsLimitRate:                c.TpsLimitRate,
+		TpsLimitStrategy:            c.TpsLimitStrategy,
+		ExecuteLimit:                c.ExecuteLimit,
+		ExecuteLimitRejectedHandler: c.ExecuteLimitRejectedHandler,
+		Sticky:                      c.Sticky,
+		RequestTimeout:              c.RequestTimeout,
+	}
+}
diff --git a/client/options.go b/client/options.go
new file mode 100644
index 0000000..c5e1760
--- /dev/null
+++ b/client/options.go
@@ -0,0 +1,433 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package client
+
+import (
+	"strconv"
+	"time"
+)
+
+import (
+	"github.com/creasty/defaults"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	commonCfg "dubbo.apache.org/dubbo-go/v3/common/config"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+	"dubbo.apache.org/dubbo-go/v3/graceful_shutdown"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/proxy"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+type ClientOptions struct {
+	Application *global.ApplicationConfig
+	Consumer    *global.ConsumerConfig
+	Reference   *global.ReferenceConfig
+	Registries  map[string]*global.RegistryConfig
+	Shutdown    *global.ShutdownConfig
+
+	pxy          *proxy.Proxy
+	id           string
+	invoker      protocol.Invoker
+	urls         []*common.URL
+	metaDataType string
+	info         *ClientInfo
+
+	methodsCompat     []*config.MethodConfig
+	applicationCompat *config.ApplicationConfig
+	registriesCompat  map[string]*config.RegistryConfig
+}
+
+func defaultClientOptions() *ClientOptions {
+	return &ClientOptions{
+		Application: global.DefaultApplicationConfig(),
+		Consumer:    global.DefaultConsumerConfig(),
+		Reference:   global.DefaultReferenceConfig(),
+		Shutdown:    global.DefaultShutdownConfig(),
+	}
+}
+
+func (cliOpts *ClientOptions) init(opts ...ClientOption) error {
+	for _, opt := range opts {
+		opt(cliOpts)
+	}
+	if err := defaults.Set(cliOpts); err != nil {
+		return err
+	}
+
+	ref := cliOpts.Reference
+
+	// init method
+	methods := ref.Methods
+	if length := len(methods); length > 0 {
+		cliOpts.methodsCompat = make([]*config.MethodConfig, length)
+		for i, method := range methods {
+			cliOpts.methodsCompat[i] = compatMethodConfig(method)
+			if err := cliOpts.methodsCompat[i].Init(); err != nil {
+				return err
+			}
+		}
+
+	}
+
+	// init application
+	application := cliOpts.Application
+	if application != nil {
+		cliOpts.applicationCompat = compatApplicationConfig(application)
+		if err := cliOpts.applicationCompat.Init(); err != nil {
+			return err
+		}
+		cliOpts.metaDataType = cliOpts.applicationCompat.MetadataType
+		if ref.Group == "" {
+			ref.Group = cliOpts.applicationCompat.Group
+		}
+		if ref.Version == "" {
+			ref.Version = cliOpts.applicationCompat.Version
+		}
+	}
+	// init cluster
+	if ref.Cluster == "" {
+		ref.Cluster = "failover"
+	}
+
+	// todo(DMwangnima): move to registry package
+	// init registries
+	var emptyRegIDsFlag bool
+	if ref.RegistryIDs == nil || len(ref.RegistryIDs) <= 0 {
+		emptyRegIDsFlag = true
+	}
+	regs := cliOpts.Registries
+	if regs != nil {
+		cliOpts.registriesCompat = make(map[string]*config.RegistryConfig)
+		for key, reg := range regs {
+			cliOpts.registriesCompat[key] = compatRegistryConfig(reg)
+			if err := cliOpts.registriesCompat[key].Init(); err != nil {
+				return err
+			}
+			if emptyRegIDsFlag {
+				ref.RegistryIDs = append(ref.RegistryIDs, key)
+			}
+		}
+	}
+	ref.RegistryIDs = commonCfg.TranslateIds(ref.RegistryIDs)
+
+	// init graceful_shutdown
+	graceful_shutdown.Init(graceful_shutdown.WithShutdown_Config(cliOpts.Shutdown))
+
+	return commonCfg.Verify(cliOpts)
+}
+
+type ClientOption func(*ClientOptions)
+
+// ---------- For user ----------
+
+func WithCheck() ClientOption {
+	return func(opts *ClientOptions) {
+		check := true
+		opts.Reference.Check = &check
+	}
+}
+
+func WithURL(url string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.URL = url
+	}
+}
+
+// todo(DMwangnima): change Filter Option like Cluster and LoadBalance
+func WithFilter(filter string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Filter = filter
+	}
+}
+
+// todo(DMwangnima): think about a more ideal configuration style
+func WithRegistryIDs(registryIDs []string) ClientOption {
+	return func(opts *ClientOptions) {
+		if len(registryIDs) > 0 {
+			opts.Reference.RegistryIDs = registryIDs
+		}
+	}
+}
+
+func WithRegistry(key string, opts ...registry.Option) ClientOption {
+	regOpts := registry.DefaultOptions()
+	for _, opt := range opts {
+		opt(regOpts)
+	}
+
+	return func(cliOpts *ClientOptions) {
+		if cliOpts.Registries == nil {
+			cliOpts.Registries = make(map[string]*global.RegistryConfig)
+		}
+		cliOpts.Registries[key] = regOpts.Registry
+	}
+}
+
+func WithShutdown(opts ...graceful_shutdown.Option) ClientOption {
+	sdOpts := graceful_shutdown.DefaultOptions()
+	for _, opt := range opts {
+		opt(sdOpts)
+	}
+
+	return func(cliOpts *ClientOptions) {
+		cliOpts.Shutdown = sdOpts.Shutdown
+	}
+}
+
+// ========== Cluster Strategy ==========
+
+func WithClusterAvailable() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyAvailable
+	}
+}
+
+func WithClusterBroadcast() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyBroadcast
+	}
+}
+
+func WithClusterFailBack() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyFailback
+	}
+}
+
+func WithClusterFailFast() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyFailfast
+	}
+}
+
+func WithClusterFailOver() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyFailover
+	}
+}
+
+func WithClusterFailSafe() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyFailsafe
+	}
+}
+
+func WithClusterForking() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyForking
+	}
+}
+
+func WithClusterZoneAware() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyZoneAware
+	}
+}
+
+func WithClusterAdaptiveService() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Cluster = constant.ClusterKeyAdaptiveService
+	}
+}
+
+// ========== LoadBalance Strategy ==========
+
+func WithLoadBalanceConsistentHashing() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyConsistentHashing
+	}
+}
+
+func WithLoadBalanceLeastActive() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyLeastActive
+	}
+}
+
+func WithLoadBalanceRandom() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyRandom
+	}
+}
+
+func WithLoadBalanceRoundRobin() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyRoundRobin
+	}
+}
+
+func WithLoadBalanceP2C() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyP2C
+	}
+}
+
+func WithLoadBalanceXDSRingHash() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Loadbalance = constant.LoadBalanceKeyLeastActive
+	}
+}
+
+func WithRetries(retries int) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Retries = strconv.Itoa(retries)
+	}
+}
+
+func WithGroup(group string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Group = group
+	}
+}
+
+func WithVersion(version string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Version = version
+	}
+}
+
+func WithJSON() ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Serialization = constant.JSONSerialization
+	}
+}
+
+func WithProvidedBy(providedBy string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.ProvidedBy = providedBy
+	}
+}
+
+// todo(DMwangnima): implement this functionality
+//func WithAsync() ClientOption {
+//	return func(opts *ClientOptions) {
+//		opts.Reference.Async = true
+//	}
+//}
+
+func WithParams(params map[string]string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Params = params
+	}
+}
+
+// todo(DMwangnima): implement this functionality
+//func WithGeneric(generic bool) ClientOption {
+//	return func(opts *ClientOptions) {
+//		if generic {
+//			opts.Reference.Generic = "true"
+//		} else {
+//			opts.Reference.Generic = "false"
+//		}
+//	}
+//}
+
+func WithSticky(sticky bool) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.Sticky = sticky
+	}
+}
+
+func WithRequestTimeout(timeout time.Duration) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.RequestTimeout = timeout.String()
+	}
+}
+
+func WithForce(force bool) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.ForceTag = force
+	}
+}
+
+func WithTracingKey(tracingKey string) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.TracingKey = tracingKey
+	}
+}
+
+func WithMeshProviderPort(port int) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference.MeshProviderPort = port
+	}
+}
+
+// ---------- For framework ----------
+// These functions should not be invoked by users
+
+func SetRegistries(regs map[string]*global.RegistryConfig) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Registries = regs
+	}
+}
+
+func SetApplication(application *global.ApplicationConfig) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Application = application
+	}
+}
+
+func SetConsumer(consumer *global.ConsumerConfig) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Consumer = consumer
+	}
+}
+
+func SetReference(reference *global.ReferenceConfig) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Reference = reference
+	}
+}
+
+func SetShutdown(shutdown *global.ShutdownConfig) ClientOption {
+	return func(opts *ClientOptions) {
+		opts.Shutdown = shutdown
+	}
+}
+
+// todo: need to be consistent with MethodConfig
+type CallOptions struct {
+	RequestTimeout string
+	Retries        string
+}
+
+type CallOption func(*CallOptions)
+
+func newDefaultCallOptions() *CallOptions {
+	return &CallOptions{
+		RequestTimeout: "",
+		Retries:        "",
+	}
+}
+
+func WithCallRequestTimeout(timeout string) CallOption {
+	return func(opts *CallOptions) {
+		opts.RequestTimeout = timeout
+	}
+}
+
+func WithCallRetries(retries string) CallOption {
+	return func(opts *CallOptions) {
+		opts.Retries = retries
+	}
+}
diff --git a/common/config/utils.go b/common/config/utils.go
new file mode 100644
index 0000000..76872fb
--- /dev/null
+++ b/common/config/utils.go
@@ -0,0 +1,99 @@
+package config
+
+import (
+	"fmt"
+	"regexp"
+	"strings"
+)
+
+import (
+	"github.com/go-playground/validator/v10"
+
+	perrors "github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+)
+
+var validate *validator.Validate
+
+func init() {
+	validate = validator.New()
+}
+
+// TranslateIds string "nacos,zk" => ["nacos","zk"]
+func TranslateIds(registryIds []string) []string {
+	ids := make([]string, 0)
+	for _, id := range registryIds {
+
+		ids = append(ids, strings.Split(id, ",")...)
+	}
+	return removeDuplicateElement(ids)
+}
+
+// removeDuplicateElement remove duplicate element
+func removeDuplicateElement(items []string) []string {
+	result := make([]string, 0, len(items))
+	temp := map[string]struct{}{}
+	for _, item := range items {
+		if _, ok := temp[item]; !ok && item != "" {
+			temp[item] = struct{}{}
+			result = append(result, item)
+		}
+	}
+	return result
+}
+
+func Verify(s interface{}) error {
+	if err := validate.Struct(s); err != nil {
+		errs := err.(validator.ValidationErrors)
+		var slice []string
+		for _, msg := range errs {
+			slice = append(slice, msg.Error())
+		}
+		return perrors.New(strings.Join(slice, ","))
+	}
+	return nil
+}
+
+func MergeValue(str1, str2, def string) string {
+	if str1 == "" && str2 == "" {
+		return def
+	}
+	s1 := strings.Split(str1, ",")
+	s2 := strings.Split(str2, ",")
+	str := "," + strings.Join(append(s1, s2...), ",")
+	defKey := strings.Contains(str, ","+constant.DefaultKey)
+	if !defKey {
+		str = "," + constant.DefaultKey + str
+	}
+	str = strings.TrimPrefix(strings.Replace(str, ","+constant.DefaultKey, ","+def, -1), ",")
+	return removeMinus(strings.Split(str, ","))
+}
+
+func removeMinus(strArr []string) string {
+	if len(strArr) == 0 {
+		return ""
+	}
+	var normalStr string
+	var minusStrArr []string
+	for _, v := range strArr {
+		if strings.HasPrefix(v, "-") {
+			minusStrArr = append(minusStrArr, v[1:])
+		} else {
+			normalStr += fmt.Sprintf(",%s", v)
+		}
+	}
+	normalStr = strings.Trim(normalStr, ",")
+	for _, v := range minusStrArr {
+		normalStr = strings.Replace(normalStr, v, "", 1)
+	}
+	reg := regexp.MustCompile("[,]+")
+	normalStr = reg.ReplaceAllString(strings.Trim(normalStr, ","), ",")
+	return normalStr
+}
+
+func IsValid(addr string) bool {
+	return addr != "" && addr != constant.NotAvailable
+}
diff --git a/common/constant/key.go b/common/constant/key.go
index 4d0b0a8..2effa04 100644
--- a/common/constant/key.go
+++ b/common/constant/key.go
@@ -136,6 +136,15 @@
 	RetryTimesKey                      = "retry.times"
 	CycleReportKey                     = "cycle.report"
 	DefaultBlackListRecoverBlock       = 16
+	CallTypeKey                        = "call-type"
+	CallUnary                          = "unary"
+	CallClientStream                   = "client-stream"
+	CallServerStream                   = "server-stream"
+	CallBidiStream                     = "bidi-stream"
+	CallHTTPTypeKey                    = "call-http-type"
+	CallHTTP                           = "http"
+	CallHTTP2                          = "http2"
+	ServiceInfoKey                     = "service-info"
 )
 
 const (
diff --git a/common/constant/serialization.go b/common/constant/serialization.go
index 620037d..d1649f5 100644
--- a/common/constant/serialization.go
+++ b/common/constant/serialization.go
@@ -26,4 +26,5 @@
 	Hessian2Serialization = "hessian2"
 	ProtobufSerialization = "protobuf"
 	MsgpackSerialization  = "msgpack"
+	JSONSerialization     = "json"
 )
diff --git a/common/url.go b/common/url.go
index 6590ebf..cd551c2 100644
--- a/common/url.go
+++ b/common/url.go
@@ -114,6 +114,8 @@
 	Username string
 	Password string
 	Methods  []string
+	// Attributes should not be transported
+	Attributes map[string]interface{} `hessian:"-"`
 	// special for registry
 	SubURL *URL
 }
@@ -218,6 +220,16 @@
 	}
 }
 
+// WithAttribute sets attribute for URL
+func WithAttribute(key string, attribute interface{}) Option {
+	return func(url *URL) {
+		if url.Attributes == nil {
+			url.Attributes = make(map[string]interface{})
+		}
+		url.Attributes[key] = attribute
+	}
+}
+
 // NewURLWithOptions will create a new URL with options
 func NewURLWithOptions(opts ...Option) *URL {
 	newURL := &URL{}
@@ -772,7 +784,8 @@
 	}
 
 	// finally execute methodConfigMergeFcn
-	for _, method := range referenceURL.Methods {
+	mergedURL.Methods = make([]string, len(referenceURL.Methods))
+	for i, method := range referenceURL.Methods {
 		for _, paramKey := range []string{constant.LoadbalanceKey, constant.ClusterKey, constant.RetriesKey, constant.TimeoutKey} {
 			if v := referenceURL.GetParam(paramKey, ""); len(v) > 0 {
 				params[paramKey] = []string{v}
@@ -784,6 +797,7 @@
 				params[methodsKey] = []string{v}
 			}
 			//}
+			mergedURL.Methods[i] = method
 		}
 	}
 	// In this way, we will raise some performance.
diff --git a/compat.go b/compat.go
new file mode 100644
index 0000000..19202bf
--- /dev/null
+++ b/compat.go
@@ -0,0 +1,371 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package dubbo
+
+import (
+	"go.uber.org/atomic"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+func compatRootConfig(c *InstanceOptions) *config.RootConfig {
+	if c == nil {
+		return nil
+	}
+	proCompat := make(map[string]*config.ProtocolConfig)
+	for k, v := range c.Protocols {
+		proCompat[k] = compatProtocolConfig(v)
+	}
+
+	regCompat := make(map[string]*config.RegistryConfig)
+	for k, v := range c.Registries {
+		regCompat[k] = compatRegistryConfig(v)
+	}
+
+	traCompat := make(map[string]*config.TracingConfig)
+	for k, v := range c.Tracing {
+		traCompat[k] = compatTracingConfig(v)
+	}
+
+	return &config.RootConfig{
+		Application:         compatApplicationConfig(c.Application),
+		Protocols:           proCompat,
+		Registries:          regCompat,
+		ConfigCenter:        compatCenterConfig(c.ConfigCenter),
+		MetadataReport:      compatMetadataReportConfig(c.MetadataReport),
+		Provider:            compatProviderConfig(c.Provider),
+		Consumer:            compatConsumerConfig(c.Consumer),
+		Metric:              compatMetricConfig(c.Metric),
+		Tracing:             traCompat,
+		Logger:              compatLoggerConfig(c.Logger),
+		Shutdown:            compatShutdownConfig(c.Shutdown),
+		EventDispatcherType: c.EventDispatcherType,
+		CacheFile:           c.CacheFile,
+		Custom:              compatCustomConfig(c.Custom),
+		Profiles:            compatProfilesConfig(c.Profiles),
+		TLSConfig:           compatTLSConfig(c.TLSConfig),
+	}
+}
+
+func compatApplicationConfig(c *global.ApplicationConfig) *config.ApplicationConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.ApplicationConfig{
+		Organization: c.Organization,
+		Name:         c.Name,
+		Module:       c.Module,
+		Group:        c.Group,
+		Version:      c.Version,
+		Owner:        c.Owner,
+		Environment:  c.Environment,
+		MetadataType: c.MetadataType,
+		Tag:          c.Tag,
+	}
+}
+
+func compatProtocolConfig(c *global.ProtocolConfig) *config.ProtocolConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.ProtocolConfig{
+		Name:                 c.Name,
+		Ip:                   c.Ip,
+		Port:                 c.Port,
+		Params:               c.Params,
+		MaxServerSendMsgSize: c.MaxServerSendMsgSize,
+		MaxServerRecvMsgSize: c.MaxServerRecvMsgSize,
+	}
+}
+
+func compatRegistryConfig(c *global.RegistryConfig) *config.RegistryConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.RegistryConfig{
+		Protocol:          c.Protocol,
+		Timeout:           c.Timeout,
+		Group:             c.Group,
+		Namespace:         c.Namespace,
+		TTL:               c.TTL,
+		Address:           c.Address,
+		Username:          c.Username,
+		Password:          c.Password,
+		Simplified:        c.Simplified,
+		Preferred:         c.Preferred,
+		Zone:              c.Zone,
+		Weight:            c.Weight,
+		Params:            c.Params,
+		RegistryType:      c.RegistryType,
+		UseAsMetaReport:   c.UseAsMetaReport,
+		UseAsConfigCenter: c.UseAsConfigCenter,
+	}
+}
+
+func compatCenterConfig(c *global.CenterConfig) *config.CenterConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.CenterConfig{
+		Protocol:      c.Protocol,
+		Address:       c.Address,
+		DataId:        c.DataId,
+		Cluster:       c.Cluster,
+		Group:         c.Group,
+		Username:      c.Username,
+		Password:      c.Password,
+		Namespace:     c.Namespace,
+		AppID:         c.AppID,
+		Timeout:       c.Timeout,
+		Params:        c.Params,
+		FileExtension: c.FileExtension,
+	}
+}
+
+func compatMetadataReportConfig(c *global.MetadataReportConfig) *config.MetadataReportConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.MetadataReportConfig{
+		Protocol:  c.Protocol,
+		Address:   c.Address,
+		Username:  c.Username,
+		Password:  c.Password,
+		Timeout:   c.Timeout,
+		Group:     c.Group,
+		Namespace: c.Namespace,
+	}
+}
+
+func compatProviderConfig(c *global.ProviderConfig) *config.ProviderConfig {
+	if c == nil {
+		return nil
+	}
+	services := make(map[string]*config.ServiceConfig)
+	for key, svc := range c.Services {
+		services[key] = compatServiceConfig(svc)
+	}
+	return &config.ProviderConfig{
+		Filter:                 c.Filter,
+		Register:               c.Register,
+		RegistryIDs:            c.RegistryIDs,
+		ProtocolIDs:            c.ProtocolIDs,
+		TracingKey:             c.TracingKey,
+		Services:               services,
+		ProxyFactory:           c.ProxyFactory,
+		FilterConf:             c.FilterConf,
+		ConfigType:             c.ConfigType,
+		AdaptiveService:        c.AdaptiveService,
+		AdaptiveServiceVerbose: c.AdaptiveServiceVerbose,
+	}
+}
+
+func compatServiceConfig(c *global.ServiceConfig) *config.ServiceConfig {
+	if c == nil {
+		return nil
+	}
+	methods := make([]*config.MethodConfig, len(c.Methods))
+	for i, method := range c.Methods {
+		methods[i] = compatMethodConfig(method)
+	}
+	protocols := make(map[string]*config.ProtocolConfig)
+	for key, pro := range c.RCProtocolsMap {
+		protocols[key] = compatProtocolConfig(pro)
+	}
+	registries := make(map[string]*config.RegistryConfig)
+	for key, reg := range c.RCRegistriesMap {
+		registries[key] = compatRegistryConfig(reg)
+	}
+	return &config.ServiceConfig{
+		Filter:                      c.Filter,
+		ProtocolIDs:                 c.ProtocolIDs,
+		Interface:                   c.Interface,
+		RegistryIDs:                 c.RegistryIDs,
+		Cluster:                     c.Cluster,
+		Loadbalance:                 c.Loadbalance,
+		Group:                       c.Group,
+		Version:                     c.Version,
+		Methods:                     methods,
+		Warmup:                      c.Warmup,
+		Retries:                     c.Retries,
+		Serialization:               c.Serialization,
+		Params:                      c.Params,
+		Token:                       c.Token,
+		AccessLog:                   c.AccessLog,
+		TpsLimiter:                  c.TpsLimiter,
+		TpsLimitInterval:            c.TpsLimitInterval,
+		TpsLimitRate:                c.TpsLimitRate,
+		TpsLimitStrategy:            c.TpsLimitStrategy,
+		TpsLimitRejectedHandler:     c.TpsLimitRejectedHandler,
+		ExecuteLimit:                c.ExecuteLimit,
+		ExecuteLimitRejectedHandler: c.ExecuteLimitRejectedHandler,
+		Auth:                        c.Auth,
+		NotRegister:                 c.NotRegister,
+		ParamSign:                   c.ParamSign,
+		Tag:                         c.Tag,
+		TracingKey:                  c.TracingKey,
+		RCProtocolsMap:              protocols,
+		RCRegistriesMap:             registries,
+		ProxyFactoryKey:             c.ProxyFactoryKey,
+	}
+}
+
+func compatMethodConfig(c *global.MethodConfig) *config.MethodConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.MethodConfig{
+		InterfaceId:                 c.InterfaceId,
+		InterfaceName:               c.InterfaceName,
+		Name:                        c.Name,
+		Retries:                     c.Retries,
+		LoadBalance:                 c.LoadBalance,
+		Weight:                      c.Weight,
+		TpsLimitInterval:            c.TpsLimitInterval,
+		TpsLimitRate:                c.TpsLimitRate,
+		TpsLimitStrategy:            c.TpsLimitStrategy,
+		ExecuteLimit:                c.ExecuteLimit,
+		ExecuteLimitRejectedHandler: c.ExecuteLimitRejectedHandler,
+		Sticky:                      c.Sticky,
+		RequestTimeout:              c.RequestTimeout,
+	}
+}
+
+func compatConsumerConfig(c *global.ConsumerConfig) *config.ConsumerConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.ConsumerConfig{
+		Filter:                         c.Filter,
+		RegistryIDs:                    c.RegistryIDs,
+		Protocol:                       c.Protocol,
+		RequestTimeout:                 c.RequestTimeout,
+		ProxyFactory:                   c.ProxyFactory,
+		Check:                          c.Check,
+		AdaptiveService:                c.AdaptiveService,
+		TracingKey:                     c.TracingKey,
+		FilterConf:                     c.FilterConf,
+		MaxWaitTimeForServiceDiscovery: c.MaxWaitTimeForServiceDiscovery,
+		MeshEnabled:                    c.MeshEnabled,
+	}
+}
+
+func compatMetricConfig(c *global.MetricConfig) *config.MetricConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.MetricConfig{
+		Mode:               c.Mode,
+		Namespace:          c.Namespace,
+		Enable:             c.Enable,
+		Port:               c.Port,
+		Path:               c.Path,
+		PushGatewayAddress: c.PushGatewayAddress,
+		SummaryMaxAge:      c.SummaryMaxAge,
+		Protocol:           c.Protocol,
+	}
+}
+
+func compatTracingConfig(c *global.TracingConfig) *config.TracingConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.TracingConfig{
+		Name:        c.Name,
+		ServiceName: c.ServiceName,
+		Address:     c.Address,
+		UseAgent:    c.UseAgent,
+	}
+}
+
+func compatLoggerConfig(c *global.LoggerConfig) *config.LoggerConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.LoggerConfig{
+		Driver:   c.Driver,
+		Level:    c.Level,
+		Format:   c.Format,
+		Appender: c.Appender,
+		File:     compatFile(c.File),
+	}
+}
+
+func compatFile(c *global.File) *config.File {
+	if c == nil {
+		return nil
+	}
+	return &config.File{
+		Name:       c.Name,
+		MaxSize:    c.MaxSize,
+		MaxBackups: c.MaxBackups,
+		MaxAge:     c.MaxAge,
+		Compress:   c.Compress,
+	}
+}
+
+func compatShutdownConfig(c *global.ShutdownConfig) *config.ShutdownConfig {
+	if c == nil {
+		return nil
+	}
+	cfg := &config.ShutdownConfig{
+		Timeout:                     c.Timeout,
+		StepTimeout:                 c.StepTimeout,
+		ConsumerUpdateWaitTime:      c.ConsumerUpdateWaitTime,
+		RejectRequestHandler:        c.RejectRequestHandler,
+		InternalSignal:              c.InternalSignal,
+		OfflineRequestWindowTimeout: c.OfflineRequestWindowTimeout,
+		RejectRequest:               atomic.Bool{},
+	}
+	cfg.RejectRequest.Store(c.RejectRequest.Load())
+
+	return cfg
+}
+
+func compatCustomConfig(c *global.CustomConfig) *config.CustomConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.CustomConfig{
+		ConfigMap: c.ConfigMap,
+	}
+}
+
+func compatProfilesConfig(c *global.ProfilesConfig) *config.ProfilesConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.ProfilesConfig{
+		Active: c.Active,
+	}
+}
+
+func compatTLSConfig(c *global.TLSConfig) *config.TLSConfig {
+	if c == nil {
+		return nil
+	}
+	return &config.TLSConfig{
+		CACertFile:    c.CACertFile,
+		TLSCertFile:   c.TLSCertFile,
+		TLSKeyFile:    c.TLSKeyFile,
+		TLSServerName: c.TLSServerName,
+	}
+}
diff --git a/config/reference_config.go b/config/reference_config.go
index e38372e..6b14313 100644
--- a/config/reference_config.go
+++ b/config/reference_config.go
@@ -225,7 +225,7 @@
 			}
 		}
 	} else { // use registry configs
-		rc.urls = loadRegistries(rc.RegistryIDs, rc.rootConfig.Registries, common.CONSUMER)
+		rc.urls = LoadRegistries(rc.RegistryIDs, rc.rootConfig.Registries, common.CONSUMER)
 		// set url to regURLs
 		for _, regURL := range rc.urls {
 			regURL.SubURL = cfgURL
diff --git a/config/registry_config.go b/config/registry_config.go
index 8948324..9eca5ea 100644
--- a/config/registry_config.go
+++ b/config/registry_config.go
@@ -208,7 +208,7 @@
 	return urls, err
 }
 
-func loadRegistries(registryIds []string, registries map[string]*RegistryConfig, roleType common.RoleType) []*common.URL {
+func LoadRegistries(registryIds []string, registries map[string]*RegistryConfig, roleType common.RoleType) []*common.URL {
 	var registryURLs []*common.URL
 	//trSlice := strings.Split(targetRegistries, ",")
 
diff --git a/config/registry_config_test.go b/config/registry_config_test.go
index fdd3b2a..83be5ed 100644
--- a/config/registry_config_test.go
+++ b/config/registry_config_test.go
@@ -43,8 +43,8 @@
 			Password: "pwd1",
 		},
 	}
-	urls := loadRegistries(target, regs, common.CONSUMER)
-	t.Logf("loadRegistries() = urls:%v", urls)
+	urls := LoadRegistries(target, regs, common.CONSUMER)
+	t.Logf("LoadRegistries() = urls:%v", urls)
 	assert.Equal(t, "127.0.0.2:2181,128.0.0.1:2181", urls[0].Location)
 	assert.Equal(t, "service-discovery-registry://127.0.0.2:2181,128.0.0.1:2181/shanghai_idc", urls[0].PrimitiveURL)
 }
@@ -62,8 +62,8 @@
 			Password: "pwd1",
 		},
 	}
-	urls := loadRegistries(target, regs, common.CONSUMER)
-	t.Logf("loadRegistries() = urls:%v", urls)
+	urls := LoadRegistries(target, regs, common.CONSUMER)
+	t.Logf("LoadRegistries() = urls:%v", urls)
 	assert.Equal(t, "127.0.0.2:2181", urls[0].Location)
 	assert.Equal(t, "service-discovery-registry://127.0.0.2:2181/shanghai_idc", urls[0].PrimitiveURL)
 }
@@ -77,7 +77,7 @@
 			RegistryType: constant.RegistryTypeAll,
 		},
 	}
-	urls := loadRegistries(target, regs, common.PROVIDER)
+	urls := LoadRegistries(target, regs, common.PROVIDER)
 	assert.Equal(t, 2, len(urls))
 	assert.Equal(t, "service-discovery-registry://127.0.0.2:2181", urls[0].PrimitiveURL)
 }
diff --git a/config/service.go b/config/service.go
index 238d554..513048a 100644
--- a/config/service.go
+++ b/config/service.go
@@ -106,3 +106,10 @@
 	}
 	return nil
 }
+
+// SetClientInfoService is used by new Triple generated code
+// use interface{} to represent info because config package can not depend on client package.
+// When refactoring work finished, this info should be with *client.ClientInfo type and this
+// function would be implemented.
+// todo(DMWangnima): refactor and implement this function
+func SetClientInfoService(info interface{}, service common.RPCService) {}
diff --git a/config/service_config.go b/config/service_config.go
index 073ad8e..7285ad8 100644
--- a/config/service_config.go
+++ b/config/service_config.go
@@ -247,7 +247,7 @@
 
 	regUrls := make([]*common.URL, 0)
 	if !s.NotRegister {
-		regUrls = loadRegistries(s.RegistryIDs, s.RCRegistriesMap, common.PROVIDER)
+		regUrls = LoadRegistries(s.RegistryIDs, s.RCRegistriesMap, common.PROVIDER)
 	}
 
 	urlMap := s.getUrlMap()
diff --git a/config/service_config_test.go b/config/service_config_test.go
index 4f5954a..a5000a4 100644
--- a/config/service_config_test.go
+++ b/config/service_config_test.go
@@ -93,8 +93,8 @@
 		assert.Equal(t, serviceConfig.IsExport(), false)
 	})
 
-	t.Run("loadRegistries&loadProtocol&getRandomPort", func(t *testing.T) {
-		registries := loadRegistries(serviceConfig.RegistryIDs, serviceConfig.RCRegistriesMap, common.PROVIDER)
+	t.Run("LoadRegistries&loadProtocol&getRandomPort", func(t *testing.T) {
+		registries := LoadRegistries(serviceConfig.RegistryIDs, serviceConfig.RCRegistriesMap, common.PROVIDER)
 		assert.Equal(t, len(registries), 1)
 		assert.Equal(t, "service-discovery-registry", registries[0].Protocol)
 		assert.Equal(t, registries[0].Port, "8848")
diff --git a/dubbo.go b/dubbo.go
new file mode 100644
index 0000000..6513653
--- /dev/null
+++ b/dubbo.go
@@ -0,0 +1,145 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package dubbo
+
+import (
+	"github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/client"
+	"dubbo.apache.org/dubbo-go/v3/global"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+// Instance is the highest layer conception that user could touch. It is mapped from RootConfig.
+// When users want to inject global configurations and configure common modules for client layer
+// and server layer, user-side code would be like this:
+//
+// ins, err := NewInstance()
+// cli, err := ins.NewClient()
+type Instance struct {
+	insOpts *InstanceOptions
+}
+
+// NewInstance receives InstanceOption and initializes RootConfig. There are some processing
+// tasks during initialization.
+func NewInstance(opts ...InstanceOption) (*Instance, error) {
+	newInsOpts := defaultInstanceOptions()
+	if err := newInsOpts.init(opts...); err != nil {
+		return nil, err
+	}
+
+	return &Instance{insOpts: newInsOpts}, nil
+}
+
+// NewClient is like client.NewClient, but inject configurations from RootConfig and
+// ConsumerConfig
+func (ins *Instance) NewClient(opts ...client.ClientOption) (*client.Client, error) {
+	if ins == nil || ins.insOpts == nil {
+		return nil, errors.New("Instance has not been initialized")
+	}
+
+	var cliOpts []client.ClientOption
+	conCfg := ins.insOpts.Consumer
+	appCfg := ins.insOpts.Application
+	regsCfg := ins.insOpts.Registries
+	sdCfg := ins.insOpts.Shutdown
+	if conCfg != nil {
+		refCfg := &global.ReferenceConfig{
+			Check:       &conCfg.Check,
+			Filter:      conCfg.Filter,
+			Protocol:    conCfg.Protocol,
+			RegistryIDs: conCfg.RegistryIDs,
+			TracingKey:  conCfg.TracingKey,
+		}
+		// these options come from Consumer and Root.
+		// for dubbo-go developers, referring config/ConsumerConfig.Init and config/ReferenceConfig
+		cliOpts = append(cliOpts,
+			client.SetReference(refCfg),
+			client.SetConsumer(conCfg),
+		)
+	}
+	if appCfg != nil {
+		cliOpts = append(cliOpts, client.SetApplication(appCfg))
+	}
+	if regsCfg != nil {
+		cliOpts = append(cliOpts, client.SetRegistries(regsCfg))
+	}
+	if sdCfg != nil {
+		cliOpts = append(cliOpts, client.SetShutdown(sdCfg))
+	}
+	// options passed by users has higher priority
+	cliOpts = append(cliOpts, opts...)
+
+	cli, err := client.NewClient(cliOpts...)
+	if err != nil {
+		return nil, err
+	}
+
+	return cli, nil
+}
+
+// NewServer is like server.NewServer, but inject configurations from RootConfig.
+func (ins *Instance) NewServer(opts ...server.ServerOption) (*server.Server, error) {
+	if ins == nil || ins.insOpts == nil {
+		return nil, errors.New("Instance has not been initialized")
+	}
+
+	var srvOpts []server.ServerOption
+	appCfg := ins.insOpts.Application
+	regsCfg := ins.insOpts.Registries
+	prosCfg := ins.insOpts.Protocols
+	trasCfg := ins.insOpts.Tracing
+	sdCfg := ins.insOpts.Shutdown
+	if appCfg != nil {
+		srvOpts = append(srvOpts,
+			server.SetServer_Application(appCfg),
+			//server.WithServer_ApplicationConfig(
+			//	global.WithApplication_Name(appCfg.Name),
+			//	global.WithApplication_Organization(appCfg.Organization),
+			//	global.WithApplication_Module(appCfg.Module),
+			//	global.WithApplication_Version(appCfg.Version),
+			//	global.WithApplication_Owner(appCfg.Owner),
+			//	global.WithApplication_Environment(appCfg.Environment),
+			//),
+		)
+	}
+	if regsCfg != nil {
+		srvOpts = append(srvOpts, server.SetServer_Registries(regsCfg))
+	}
+	if prosCfg != nil {
+		srvOpts = append(srvOpts, server.SetServer_Protocols(prosCfg))
+	}
+	if trasCfg != nil {
+		srvOpts = append(srvOpts, server.SetServer_Tracings(trasCfg))
+	}
+	if sdCfg != nil {
+		srvOpts = append(srvOpts, server.SetServer_Shutdown(sdCfg))
+	}
+
+	// options passed by users have higher priority
+	srvOpts = append(srvOpts, opts...)
+
+	srv, err := server.NewServer(srvOpts...)
+	if err != nil {
+		return nil, err
+	}
+
+	return srv, nil
+}
diff --git a/global/application_config.go b/global/application_config.go
new file mode 100644
index 0000000..9a197e7
--- /dev/null
+++ b/global/application_config.go
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// ApplicationConfig is a configuration for current applicationConfig, whether the applicationConfig is a provider or a consumer
+type ApplicationConfig struct {
+	Organization string `default:"dubbo-go" yaml:"organization" json:"organization,omitempty" property:"organization"`
+	Name         string `default:"dubbo.io" yaml:"name" json:"name,omitempty" property:"name"`
+	Module       string `default:"sample" yaml:"module" json:"module,omitempty" property:"module"`
+	Group        string `yaml:"group" json:"group,omitempty" property:"module"`
+	Version      string `yaml:"version" json:"version,omitempty" property:"version"`
+	Owner        string `default:"dubbo-go" yaml:"owner" json:"owner,omitempty" property:"owner"`
+	Environment  string `yaml:"environment" json:"environment,omitempty" property:"environment"`
+	// the metadata type. remote or local
+	MetadataType string `default:"local" yaml:"metadata-type" json:"metadataType,omitempty" property:"metadataType"`
+	Tag          string `yaml:"tag" json:"tag,omitempty" property:"tag"`
+}
+
+func DefaultApplicationConfig() *ApplicationConfig {
+	// return a new config without setting any field means there is not any default value for initialization
+	return &ApplicationConfig{}
+}
+
+type ApplicationOption func(*ApplicationConfig)
+
+func WithApplication_Organization(organization string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Organization = organization
+	}
+}
+
+func WithApplication_Name(name string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Name = name
+	}
+}
+
+func WithApplication_Module(module string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Module = module
+	}
+}
+
+func WithApplication_Group(group string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithApplication_Version(version string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Version = version
+	}
+}
+
+func WithApplication_Owner(owner string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Owner = owner
+	}
+}
+
+func WithApplication_Environment(environment string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Environment = environment
+	}
+}
+
+func WithApplication_MetadataType(metadataType string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.MetadataType = metadataType
+	}
+}
+
+func WithApplication_Tag(tag string) ApplicationOption {
+	return func(cfg *ApplicationConfig) {
+		cfg.Tag = tag
+	}
+}
diff --git a/global/config_center_config.go b/global/config_center_config.go
new file mode 100644
index 0000000..d219b69
--- /dev/null
+++ b/global/config_center_config.go
@@ -0,0 +1,110 @@
+package global
+
+// CenterConfig is configuration for config center
+//
+// ConfigCenter also introduced concepts of namespace and group to better manage Key-Value pairs by group,
+// those configs are already built-in in many professional third-party configuration centers.
+// In most cases, namespace is used to isolate different tenants, while group is used to divide the key set from one tenant into groups.
+//
+// CenterConfig has currently supported Zookeeper, Nacos, Etcd, Consul, Apollo
+type CenterConfig struct {
+	Protocol  string            `validate:"required" yaml:"protocol"  json:"protocol,omitempty"`
+	Address   string            `validate:"required" yaml:"address" json:"address,omitempty"`
+	DataId    string            `yaml:"data-id" json:"data-id,omitempty"`
+	Cluster   string            `yaml:"cluster" json:"cluster,omitempty"`
+	Group     string            `yaml:"group" json:"group,omitempty"`
+	Username  string            `yaml:"username" json:"username,omitempty"`
+	Password  string            `yaml:"password" json:"password,omitempty"`
+	Namespace string            `yaml:"namespace"  json:"namespace,omitempty"`
+	AppID     string            `default:"dubbo" yaml:"app-id"  json:"app-id,omitempty"`
+	Timeout   string            `default:"10s" yaml:"timeout"  json:"timeout,omitempty"`
+	Params    map[string]string `yaml:"params"  json:"parameters,omitempty"`
+
+	//FileExtension the suffix of config dataId, also the file extension of config content
+	FileExtension string `default:"yaml" yaml:"file-extension" json:"file-extension" `
+}
+
+func DefaultCenterConfig() *CenterConfig {
+	return &CenterConfig{
+		Params: make(map[string]string),
+	}
+}
+
+type CenterOption func(*CenterConfig)
+
+func WithCenter_Protocol(protocol string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Protocol = protocol
+	}
+}
+
+func WithCenter_Address(address string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Address = address
+	}
+}
+
+func WithCenter_DataID(id string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.DataId = id
+	}
+}
+
+func WithCenter_Cluster(cluster string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Cluster = cluster
+	}
+}
+
+func WithCenter_Group(group string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithCenter_Username(name string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Username = name
+	}
+}
+
+func WithCenter_Password(password string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Password = password
+	}
+}
+
+func WithCenter_Namespace(namespace string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Namespace = namespace
+	}
+}
+
+func WithCenter_AppID(id string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.AppID = id
+	}
+}
+
+func WithCenter_Timeout(timeout string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.Timeout = timeout
+	}
+}
+
+func WithCenter_Params(params map[string]string) CenterOption {
+	return func(cfg *CenterConfig) {
+		if cfg.Params == nil {
+			cfg.Params = make(map[string]string)
+		}
+		for k, v := range params {
+			cfg.Params[k] = v
+		}
+	}
+}
+
+func WithCenter_FileExtension(extension string) CenterOption {
+	return func(cfg *CenterConfig) {
+		cfg.FileExtension = extension
+	}
+}
diff --git a/global/consumer_config.go b/global/consumer_config.go
new file mode 100644
index 0000000..d866e5d
--- /dev/null
+++ b/global/consumer_config.go
@@ -0,0 +1,109 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+type ConsumerConfig struct {
+	Filter          string   `yaml:"filter" json:"filter,omitempty" property:"filter"`
+	RegistryIDs     []string `yaml:"registry-ids" json:"registry-ids,omitempty" property:"registry-ids"`
+	Protocol        string   `yaml:"protocol" json:"protocol,omitempty" property:"protocol"`
+	RequestTimeout  string   `default:"3s" yaml:"request-timeout" json:"request-timeout,omitempty" property:"request-timeout"`
+	ProxyFactory    string   `default:"default" yaml:"proxy" json:"proxy,omitempty" property:"proxy"`
+	Check           bool     `yaml:"check" json:"check,omitempty" property:"check"`
+	AdaptiveService bool     `default:"false" yaml:"adaptive-service" json:"adaptive-service" property:"adaptive-service"`
+	// there is no need to configure References, it will be replaced by instance.NewClient
+	//References                     map[string]*client.ReferenceConfig `yaml:"references" json:"references,omitempty" property:"references"`
+	TracingKey                     string      `yaml:"tracing-key" json:"tracing-key" property:"tracing-key"`
+	FilterConf                     interface{} `yaml:"filter-conf" json:"filter-conf,omitempty" property:"filter-conf"`
+	MaxWaitTimeForServiceDiscovery string      `default:"3s" yaml:"max-wait-time-for-service-discovery" json:"max-wait-time-for-service-discovery,omitempty" property:"max-wait-time-for-service-discovery"`
+	MeshEnabled                    bool        `yaml:"mesh-enabled" json:"mesh-enabled,omitempty" property:"mesh-enabled"`
+}
+
+func DefaultConsumerConfig() *ConsumerConfig {
+	return &ConsumerConfig{
+		RequestTimeout: "3s",
+		Check:          true,
+	}
+}
+
+type ConsumerOption func(*ConsumerConfig)
+
+func WithConsumer_Filter(filter string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.Filter = filter
+	}
+}
+
+func WithConsumer_RegistryIDs(ids []string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.RegistryIDs = ids
+	}
+}
+
+func WithConsumer_Protocol(protocol string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.Protocol = protocol
+	}
+}
+
+func WithConsumer_RequestTimeout(timeout string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.RequestTimeout = timeout
+	}
+}
+
+func WithConsumer_ProxyFactory(factory string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.ProxyFactory = factory
+	}
+}
+
+func WithConsumer_Check(flag bool) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.Check = flag
+	}
+}
+
+func WithConsumer_AdaptiveService(flag bool) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.AdaptiveService = flag
+	}
+}
+
+func WithConsumer_TracingKey(key string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.TracingKey = key
+	}
+}
+
+func WithConsumer_FilterConf(conf interface{}) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.FilterConf = conf
+	}
+}
+
+func WithConsumer_MaxWaitTimeForServiceDiscovery(time string) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.MaxWaitTimeForServiceDiscovery = time
+	}
+}
+
+func WithConsumer_MeshEnabled(flag bool) ConsumerOption {
+	return func(cfg *ConsumerConfig) {
+		cfg.MeshEnabled = flag
+	}
+}
diff --git a/global/custom_config.go b/global/custom_config.go
new file mode 100644
index 0000000..e35e406
--- /dev/null
+++ b/global/custom_config.go
@@ -0,0 +1,34 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+type CustomConfig struct {
+	ConfigMap map[string]interface{} `yaml:"config-map" json:"config-map,omitempty" property:"config-map"`
+}
+
+func DefaultCustomConfig() *CustomConfig {
+	return &CustomConfig{}
+}
+
+type CustomOption func(*CustomConfig)
+
+func WithCustom_ConfigMap(cfgMap map[string]interface{}) CustomOption {
+	return func(cfg *CustomConfig) {
+		cfg.ConfigMap = cfgMap
+	}
+}
diff --git a/global/logger_config.go b/global/logger_config.go
new file mode 100644
index 0000000..f55ba27
--- /dev/null
+++ b/global/logger_config.go
@@ -0,0 +1,136 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+import (
+	"github.com/creasty/defaults"
+)
+
+type LoggerConfig struct {
+	// logger driver default zap
+	Driver string `default:"zap" yaml:"driver"`
+
+	// logger level
+	Level string `default:"info" yaml:"level"`
+
+	// logger formatter default text
+	Format string `default:"text" yaml:"format"`
+
+	// supports simultaneous file and console eg: console,file default console
+	Appender string `default:"console" yaml:"appender"`
+
+	// logger file
+	File *File `yaml:"file"`
+}
+
+type File struct {
+	// log file name default dubbo.log
+	Name string `default:"dubbo.log" yaml:"name"`
+
+	// log max size default 100Mb
+	MaxSize int `default:"100" yaml:"max-size"`
+
+	// log max backups default 5
+	MaxBackups int `default:"5" yaml:"max-backups"`
+
+	// log file max age default 3 day
+	MaxAge int `default:"3" yaml:"max-age"`
+
+	Compress *bool `default:"true" yaml:"compress"`
+}
+
+func DefaultLoggerConfig() *LoggerConfig {
+	// this logic is same as /config/logger_config.go/LoggerConfigBuilder.Build
+	cfg := &LoggerConfig{
+		File: &File{},
+	}
+	defaults.MustSet(cfg)
+
+	return cfg
+}
+
+type LoggerOption func(*LoggerConfig)
+
+func WithLogger_Driver(driver string) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		cfg.Driver = driver
+	}
+}
+
+func WithLogger_Level(level string) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		cfg.Level = level
+	}
+}
+
+func WithLogger_Format(format string) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		cfg.Format = format
+	}
+}
+
+func WithLogger_Appender(appender string) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		cfg.Appender = appender
+	}
+}
+
+func WithLogger_File_Name(name string) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		if cfg.File == nil {
+			cfg.File = new(File)
+		}
+		cfg.File.Name = name
+	}
+}
+
+func WithLogger_File_MaxSize(size int) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		if cfg.File == nil {
+			cfg.File = new(File)
+		}
+		cfg.File.MaxSize = size
+	}
+}
+
+func WithLogger_File_MaxBackups(backups int) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		if cfg.File == nil {
+			cfg.File = new(File)
+		}
+		cfg.File.MaxBackups = backups
+	}
+}
+
+func WithLogger_File_MaxAge(age int) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		if cfg.File == nil {
+			cfg.File = new(File)
+		}
+		cfg.File.MaxAge = age
+	}
+}
+
+func WithLogger_File_Compress(flag bool) LoggerOption {
+	return func(cfg *LoggerConfig) {
+		if cfg.File == nil {
+			cfg.File = new(File)
+		}
+		cfg.File.Compress = &flag
+	}
+}
diff --git a/global/metadata_report_config.go b/global/metadata_report_config.go
new file mode 100644
index 0000000..65c97f6
--- /dev/null
+++ b/global/metadata_report_config.go
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// MetadataReportConfig is app level configuration
+type MetadataReportConfig struct {
+	Protocol  string `required:"true"  yaml:"protocol"  json:"protocol,omitempty"`
+	Address   string `required:"true" yaml:"address" json:"address"`
+	Username  string `yaml:"username" json:"username,omitempty"`
+	Password  string `yaml:"password" json:"password,omitempty"`
+	Timeout   string `yaml:"timeout" json:"timeout,omitempty"`
+	Group     string `yaml:"group" json:"group,omitempty"`
+	Namespace string `yaml:"namespace" json:"namespace,omitempty"`
+	// metadataType of this application is defined by application config, local or remote
+	metadataType string
+}
+
+func DefaultMetadataReportConfig() *MetadataReportConfig {
+	// return a new config without setting any field means there is not any default value for initialization
+	return &MetadataReportConfig{}
+}
+
+type MetadataReportOption func(*MetadataReportConfig)
+
+func WithMetadataReport_Protocol(protocol string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Protocol = protocol
+	}
+}
+
+func WithMetadataReport_Address(address string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Address = address
+	}
+}
+
+func WithMetadataReport_Username(username string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Username = username
+	}
+}
+
+func WithMetadataReport_Password(password string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Password = password
+	}
+}
+
+func WithMetadataReport_Timeout(timeout string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Timeout = timeout
+	}
+}
+
+func WithMetadataReport_Group(group string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithMetadataReport_Namespace(namespace string) MetadataReportOption {
+	return func(cfg *MetadataReportConfig) {
+		cfg.Namespace = namespace
+	}
+}
diff --git a/global/method_config.go b/global/method_config.go
new file mode 100644
index 0000000..2090a3e
--- /dev/null
+++ b/global/method_config.go
@@ -0,0 +1,18 @@
+package global
+
+// MethodConfig defines method config
+type MethodConfig struct {
+	InterfaceId                 string
+	InterfaceName               string
+	Name                        string `yaml:"name"  json:"name,omitempty" property:"name"`
+	Retries                     string `yaml:"retries"  json:"retries,omitempty" property:"retries"`
+	LoadBalance                 string `yaml:"loadbalance"  json:"loadbalance,omitempty" property:"loadbalance"`
+	Weight                      int64  `yaml:"weight"  json:"weight,omitempty" property:"weight"`
+	TpsLimitInterval            string `yaml:"tps.limit.interval" json:"tps.limit.interval,omitempty" property:"tps.limit.interval"`
+	TpsLimitRate                string `yaml:"tps.limit.rate" json:"tps.limit.rate,omitempty" property:"tps.limit.rate"`
+	TpsLimitStrategy            string `yaml:"tps.limit.strategy" json:"tps.limit.strategy,omitempty" property:"tps.limit.strategy"`
+	ExecuteLimit                string `yaml:"execute.limit" json:"execute.limit,omitempty" property:"execute.limit"`
+	ExecuteLimitRejectedHandler string `yaml:"execute.limit.rejected.handler" json:"execute.limit.rejected.handler,omitempty" property:"execute.limit.rejected.handler"`
+	Sticky                      bool   `yaml:"sticky"   json:"sticky,omitempty" property:"sticky"`
+	RequestTimeout              string `yaml:"timeout"  json:"timeout,omitempty" property:"timeout"`
+}
diff --git a/global/metric_config.go b/global/metric_config.go
new file mode 100644
index 0000000..17df20a
--- /dev/null
+++ b/global/metric_config.go
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// MetricConfig This is the config struct for all metrics implementation
+type MetricConfig struct {
+	Mode               string `default:"pull" yaml:"mode" json:"mode,omitempty" property:"mode"` // push or pull,
+	Namespace          string `default:"dubbo" yaml:"namespace" json:"namespace,omitempty" property:"namespace"`
+	Enable             *bool  `default:"false" yaml:"enable" json:"enable,omitempty" property:"enable"`
+	Port               string `default:"9090" yaml:"port" json:"port,omitempty" property:"port"`
+	Path               string `default:"/metrics" yaml:"path" json:"path,omitempty" property:"path"`
+	PushGatewayAddress string `default:"" yaml:"push-gateway-address" json:"push-gateway-address,omitempty" property:"push-gateway-address"`
+	SummaryMaxAge      int64  `default:"600000000000" yaml:"summary-max-age" json:"summary-max-age,omitempty" property:"summary-max-age"`
+	Protocol           string `default:"prometheus" yaml:"protocol" json:"protocol,omitempty" property:"protocol"`
+}
+
+func DefaultMetricConfig() *MetricConfig {
+	// return a new config without setting any field means there is not any default value for initialization
+	return &MetricConfig{}
+}
+
+type MetricOption func(*MetricConfig)
+
+func WithMetric_Mode(mode string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Mode = mode
+	}
+}
+
+func WithMetric_Namespace(namespace string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Namespace = namespace
+	}
+}
+
+func WithMetric_Enable(enable bool) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Enable = &enable
+	}
+}
+
+func WithMetric_Port(port string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Port = port
+	}
+}
+
+func WithMetric_Path(path string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Path = path
+	}
+}
+
+func WithMetric_PushGatewayAddress(address string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.PushGatewayAddress = address
+	}
+}
+
+func WithMetric_SummaryMaxAge(age int64) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.SummaryMaxAge = age
+	}
+}
+
+func WithMetric_Protocol(protocol string) MetricOption {
+	return func(cfg *MetricConfig) {
+		cfg.Protocol = protocol
+	}
+}
diff --git a/global/profiles_config.go b/global/profiles_config.go
new file mode 100644
index 0000000..d829dd8
--- /dev/null
+++ b/global/profiles_config.go
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+type ProfilesConfig struct {
+	// active profiles
+	Active string
+}
+
+func DefaultProfilesConfig() *ProfilesConfig {
+	return &ProfilesConfig{}
+}
+
+type ProfilesOption func(*ProfilesConfig)
+
+func WithProfiles_Active(active string) ProfilesOption {
+	return func(cfg *ProfilesConfig) {
+		cfg.Active = active
+	}
+}
diff --git a/global/protocol_config.go b/global/protocol_config.go
new file mode 100644
index 0000000..25f1ae2
--- /dev/null
+++ b/global/protocol_config.go
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// ProtocolConfig is protocol configuration
+type ProtocolConfig struct {
+	Name   string      `default:"dubbo" validate:"required" yaml:"name" json:"name,omitempty" property:"name"`
+	Ip     string      `yaml:"ip"  json:"ip,omitempty" property:"ip"`
+	Port   string      `default:"20000" yaml:"port" json:"port,omitempty" property:"port"`
+	Params interface{} `yaml:"params" json:"params,omitempty" property:"params"`
+
+	// MaxServerSendMsgSize max size of server send message, 1mb=1000kb=1000000b 1mib=1024kb=1048576b.
+	// more detail to see https://pkg.go.dev/github.com/dustin/go-humanize#pkg-constants
+	MaxServerSendMsgSize string `yaml:"max-server-send-msg-size" json:"max-server-send-msg-size,omitempty"`
+	// MaxServerRecvMsgSize max size of server receive message
+	MaxServerRecvMsgSize string `default:"4mib" yaml:"max-server-recv-msg-size" json:"max-server-recv-msg-size,omitempty"`
+}
+
+func DefaultProtocolConfig() *ProtocolConfig {
+	return &ProtocolConfig{}
+}
+
+type ProtocolOption func(*ProtocolConfig)
+
+func WithProtocol_Name(name string) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.Name = name
+	}
+}
+
+func WithProtocol_Ip(ip string) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.Ip = ip
+	}
+}
+
+func WithProtocol_Port(port string) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.Port = port
+	}
+}
+
+func WithProtocol_Params(params interface{}) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.Params = params
+	}
+}
+
+func WithProtocol_MaxServerSendMsgSize(size string) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.MaxServerSendMsgSize = size
+	}
+}
+
+func WithProtocol_MaxServerRecvMsgSize(size string) ProtocolOption {
+	return func(cfg *ProtocolConfig) {
+		cfg.MaxServerRecvMsgSize = size
+	}
+}
diff --git a/global/provider_config.go b/global/provider_config.go
new file mode 100644
index 0000000..ffba16e
--- /dev/null
+++ b/global/provider_config.go
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// ProviderConfig is the default configuration of service provider
+type ProviderConfig struct {
+	Filter string `yaml:"filter" json:"filter,omitempty" property:"filter"`
+	// Deprecated Register whether registration is required
+	Register bool `yaml:"register" json:"register" property:"register"`
+	// RegistryIDs is registry ids list
+	RegistryIDs []string `yaml:"registry-ids" json:"registry-ids" property:"registry-ids"`
+	// protocol
+	ProtocolIDs []string `yaml:"protocol-ids" json:"protocol-ids" property:"protocol-ids"`
+	// TracingKey is tracing ids list
+	TracingKey string `yaml:"tracing-key" json:"tracing-key" property:"tracing-key"`
+	// there is no need to configure Services
+	// Services services
+	Services     map[string]*ServiceConfig `yaml:"services" json:"services,omitempty" property:"services"`
+	ProxyFactory string                    `default:"default" yaml:"proxy" json:"proxy,omitempty" property:"proxy"`
+	FilterConf   interface{}               `yaml:"filter_conf" json:"filter_conf,omitempty" property:"filter_conf"`
+	ConfigType   map[string]string         `yaml:"config_type" json:"config_type,omitempty" property:"config_type"`
+	// adaptive service
+	AdaptiveService        bool `yaml:"adaptive-service" json:"adaptive-service" property:"adaptive-service"`
+	AdaptiveServiceVerbose bool `yaml:"adaptive-service-verbose" json:"adaptive-service-verbose" property:"adaptive-service-verbose"`
+}
+
+func DefaultProviderConfig() *ProviderConfig {
+	return &ProviderConfig{
+		RegistryIDs: make([]string, 8),
+		ProtocolIDs: make([]string, 8),
+		Services:    make(map[string]*ServiceConfig),
+	}
+}
+
+type ProviderOption func(*ProviderConfig)
+
+func WithProvider_Filter(filter string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.Filter = filter
+	}
+}
+
+func WithProvider_Register(flag bool) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.Register = flag
+	}
+}
+
+func WithProvider_RegistryIDs(ids []string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.RegistryIDs = ids
+	}
+}
+
+func WithProvider_ProtocolIDs(ids []string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.ProtocolIDs = ids
+	}
+}
+
+func WithProvider_TracingKey(key string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.TracingKey = key
+	}
+}
+
+func WithProvider_ProxyFactory(factory string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.ProxyFactory = factory
+	}
+}
+
+func WithProvider_FilterConf(conf []interface{}) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.FilterConf = conf
+	}
+}
+
+func WithProvider_ConfigType(typ map[string]string) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.ConfigType = typ
+	}
+}
+
+func WithProvider_AdaptiveService(flag bool) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.AdaptiveService = flag
+	}
+}
+
+func WithProvider_AdaptiveServiceVerbose(flag bool) ProviderOption {
+	return func(cfg *ProviderConfig) {
+		cfg.AdaptiveServiceVerbose = flag
+	}
+}
diff --git a/global/reference_config.go b/global/reference_config.go
new file mode 100644
index 0000000..94070fd
--- /dev/null
+++ b/global/reference_config.go
@@ -0,0 +1,187 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+import (
+	"strconv"
+)
+
+// ReferenceConfig is the configuration of service consumer
+type ReferenceConfig struct {
+	InterfaceName    string            `yaml:"interface"  json:"interface,omitempty" property:"interface"`
+	Check            *bool             `yaml:"check"  json:"check,omitempty" property:"check"`
+	URL              string            `yaml:"url"  json:"url,omitempty" property:"url"`
+	Filter           string            `yaml:"filter" json:"filter,omitempty" property:"filter"`
+	Protocol         string            `yaml:"protocol"  json:"protocol,omitempty" property:"protocol"`
+	RegistryIDs      []string          `yaml:"registry-ids"  json:"registry-ids,omitempty"  property:"registry-ids"`
+	Cluster          string            `yaml:"cluster"  json:"cluster,omitempty" property:"cluster"`
+	Loadbalance      string            `yaml:"loadbalance"  json:"loadbalance,omitempty" property:"loadbalance"`
+	Retries          string            `yaml:"retries"  json:"retries,omitempty" property:"retries"`
+	Group            string            `yaml:"group"  json:"group,omitempty" property:"group"`
+	Version          string            `yaml:"version"  json:"version,omitempty" property:"version"`
+	Serialization    string            `yaml:"serialization" json:"serialization" property:"serialization"`
+	ProvidedBy       string            `yaml:"provided_by"  json:"provided_by,omitempty" property:"provided_by"`
+	Methods          []*MethodConfig   `yaml:"methods"  json:"methods,omitempty" property:"methods"`
+	Async            bool              `yaml:"async"  json:"async,omitempty" property:"async"`
+	Params           map[string]string `yaml:"params"  json:"params,omitempty" property:"params"`
+	Generic          string            `yaml:"generic"  json:"generic,omitempty" property:"generic"`
+	Sticky           bool              `yaml:"sticky"   json:"sticky,omitempty" property:"sticky"`
+	RequestTimeout   string            `yaml:"timeout"  json:"timeout,omitempty" property:"timeout"`
+	ForceTag         bool              `yaml:"force.tag"  json:"force.tag,omitempty" property:"force.tag"`
+	TracingKey       string            `yaml:"tracing-key" json:"tracing-key,omitempty" propertiy:"tracing-key"`
+	MeshProviderPort int               `yaml:"mesh-provider-port" json:"mesh-provider-port,omitempty" propertiy:"mesh-provider-port"`
+}
+
+func DefaultReferenceConfig() *ReferenceConfig {
+	return &ReferenceConfig{
+		// use Triple protocol by default
+		Protocol: "tri",
+		Methods:  make([]*MethodConfig, 0, 8),
+		Params:   make(map[string]string, 8),
+	}
+}
+
+type ReferenceOption func(*ReferenceConfig)
+
+func WithReference_InterfaceName(name string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.InterfaceName = name
+	}
+}
+
+func WithReference_Check(check bool) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Check = &check
+	}
+}
+
+func WithReference_URL(url string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.URL = url
+	}
+}
+
+func WithReference_Filter(filter string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Filter = filter
+	}
+}
+
+func WithReference_Protocol(protocol string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Protocol = protocol
+	}
+}
+
+func WithReference_RegistryIDs(registryIDs []string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		if len(registryIDs) >= 0 {
+			cfg.RegistryIDs = registryIDs
+		}
+	}
+}
+
+func WithReference_Cluster(cluster string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Cluster = cluster
+	}
+}
+
+func WithReference_LoadBalance(loadBalance string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Loadbalance = loadBalance
+	}
+}
+
+func WithReference_Retries(retries int) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Retries = strconv.Itoa(retries)
+	}
+}
+
+func WithReference_Group(group string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithReference_Version(version string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Version = version
+	}
+}
+
+func WithReference_Serialization(serialization string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Serialization = serialization
+	}
+}
+
+func WithReference_ProviderBy(providedBy string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.ProvidedBy = providedBy
+	}
+}
+
+func WithReference_Async(async bool) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Async = async
+	}
+}
+
+func WithReference_Params(params map[string]string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Params = params
+	}
+}
+
+func WithReference_Generic(generic string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Generic = generic
+	}
+}
+
+func WithReference_Sticky(sticky bool) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.Sticky = sticky
+	}
+}
+
+func WithReference_RequestTimeout(timeout string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.RequestTimeout = timeout
+	}
+}
+
+func WithReference_Force(force bool) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.ForceTag = force
+	}
+}
+
+func WithReference_TracingKey(tracingKey string) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.TracingKey = tracingKey
+	}
+}
+
+func WithReference_MeshProviderPort(port int) ReferenceOption {
+	return func(cfg *ReferenceConfig) {
+		cfg.MeshProviderPort = port
+	}
+}
diff --git a/global/registry_config.go b/global/registry_config.go
new file mode 100644
index 0000000..f96ff52
--- /dev/null
+++ b/global/registry_config.go
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// todo(DMwangnima): finish refactoring and move related logic from config package to this file.
+// This RegistryConfig is a copy of /config/RegistryConfig right now.
+// Please refer to issue(https://github.com/apache/dubbo-go/issues/2377).
+
+// RegistryConfig is the configuration of the registry center
+type RegistryConfig struct {
+	Protocol          string            `validate:"required" yaml:"protocol"  json:"protocol,omitempty" property:"protocol"`
+	Timeout           string            `default:"5s" validate:"required" yaml:"timeout" json:"timeout,omitempty" property:"timeout"` // unit: second
+	Group             string            `yaml:"group" json:"group,omitempty" property:"group"`
+	Namespace         string            `yaml:"namespace" json:"namespace,omitempty" property:"namespace"`
+	TTL               string            `default:"15m" yaml:"ttl" json:"ttl,omitempty" property:"ttl"` // unit: minute
+	Address           string            `validate:"required" yaml:"address" json:"address,omitempty" property:"address"`
+	Username          string            `yaml:"username" json:"username,omitempty" property:"username"`
+	Password          string            `yaml:"password" json:"password,omitempty"  property:"password"`
+	Simplified        bool              `yaml:"simplified" json:"simplified,omitempty"  property:"simplified"`
+	Preferred         bool              `yaml:"preferred" json:"preferred,omitempty" property:"preferred"` // Always use this registry first if set to true, useful when subscribe to multiple registriesConfig
+	Zone              string            `yaml:"zone" json:"zone,omitempty" property:"zone"`                // The region where the registry belongs, usually used to isolate traffics
+	Weight            int64             `yaml:"weight" json:"weight,omitempty" property:"weight"`          // Affects traffic distribution among registriesConfig, useful when subscribe to multiple registriesConfig Take effect only when no preferred registry is specified.
+	Params            map[string]string `yaml:"params" json:"params,omitempty" property:"params"`
+	RegistryType      string            `yaml:"registry-type"`
+	UseAsMetaReport   bool              `default:"true" yaml:"use-as-meta-report" json:"use-as-meta-report,omitempty" property:"use-as-meta-report"`
+	UseAsConfigCenter bool              `default:"true" yaml:"use-as-config-center" json:"use-as-config-center,omitempty" property:"use-as-config-center"`
+}
+
+func DefaultRegistryConfig() *RegistryConfig {
+	return &RegistryConfig{}
+}
+
+type RegistryOption func(*RegistryConfig)
+
+func WithRegistry_Protocol(protocol string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Protocol = protocol
+	}
+}
+
+func WithRegistry_Timeout(timeout string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Timeout = timeout
+	}
+}
+
+func WithRegistry_Group(group string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithRegistry_Namespace(namespace string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Namespace = namespace
+	}
+}
+
+func WithRegistry_TTL(ttl string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.TTL = ttl
+	}
+}
+
+func WithRegistry_Address(address string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Address = address
+	}
+}
+
+func WithRegistry_Username(name string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Username = name
+	}
+}
+
+func WithRegistry_Password(password string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Password = password
+	}
+}
+
+func WithRegistry_Simplified(flag bool) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Simplified = flag
+	}
+}
+
+func WithRegistry_Preferred(flag bool) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Preferred = flag
+	}
+}
+
+func WithRegistry_Zone(zone string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Zone = zone
+	}
+}
+
+func WithRegistry_Weight(weight int64) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Weight = weight
+	}
+}
+
+func WithRegistry_Params(params map[string]string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.Params = params
+	}
+}
+
+func WithRegistry_RegistryType(typ string) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.RegistryType = typ
+	}
+}
+
+func WithRegistry_UseAsMetaReport(flag bool) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.UseAsMetaReport = flag
+	}
+}
+
+func WithRegistry_UseAsConfigCenter(flag bool) RegistryOption {
+	return func(cfg *RegistryConfig) {
+		cfg.UseAsConfigCenter = flag
+	}
+}
diff --git a/global/service_config.go b/global/service_config.go
new file mode 100644
index 0000000..95d271d
--- /dev/null
+++ b/global/service_config.go
@@ -0,0 +1,237 @@
+package global
+
+// ServiceConfig is the configuration of the service provider
+type ServiceConfig struct {
+	Filter                      string            `yaml:"filter" json:"filter,omitempty" property:"filter"`
+	ProtocolIDs                 []string          `yaml:"protocol-ids"  json:"protocol-ids,omitempty" property:"protocol-ids"` // multi protocolIDs support, split by ','
+	Interface                   string            `yaml:"interface"  json:"interface,omitempty" property:"interface"`
+	RegistryIDs                 []string          `yaml:"registry-ids"  json:"registry-ids,omitempty"  property:"registry-ids"`
+	Cluster                     string            `default:"failover" yaml:"cluster"  json:"cluster,omitempty" property:"cluster"`
+	Loadbalance                 string            `default:"random" yaml:"loadbalance"  json:"loadbalance,omitempty"  property:"loadbalance"`
+	Group                       string            `yaml:"group"  json:"group,omitempty" property:"group"`
+	Version                     string            `yaml:"version"  json:"version,omitempty" property:"version" `
+	Methods                     []*MethodConfig   `yaml:"methods"  json:"methods,omitempty" property:"methods"`
+	Warmup                      string            `yaml:"warmup"  json:"warmup,omitempty"  property:"warmup"`
+	Retries                     string            `yaml:"retries"  json:"retries,omitempty" property:"retries"`
+	Serialization               string            `yaml:"serialization" json:"serialization" property:"serialization"`
+	Params                      map[string]string `yaml:"params"  json:"params,omitempty" property:"params"`
+	Token                       string            `yaml:"token" json:"token,omitempty" property:"token"`
+	AccessLog                   string            `yaml:"accesslog" json:"accesslog,omitempty" property:"accesslog"`
+	TpsLimiter                  string            `yaml:"tps.limiter" json:"tps.limiter,omitempty" property:"tps.limiter"`
+	TpsLimitInterval            string            `yaml:"tps.limit.interval" json:"tps.limit.interval,omitempty" property:"tps.limit.interval"`
+	TpsLimitRate                string            `yaml:"tps.limit.rate" json:"tps.limit.rate,omitempty" property:"tps.limit.rate"`
+	TpsLimitStrategy            string            `yaml:"tps.limit.strategy" json:"tps.limit.strategy,omitempty" property:"tps.limit.strategy"`
+	TpsLimitRejectedHandler     string            `yaml:"tps.limit.rejected.handler" json:"tps.limit.rejected.handler,omitempty" property:"tps.limit.rejected.handler"`
+	ExecuteLimit                string            `yaml:"execute.limit" json:"execute.limit,omitempty" property:"execute.limit"`
+	ExecuteLimitRejectedHandler string            `yaml:"execute.limit.rejected.handler" json:"execute.limit.rejected.handler,omitempty" property:"execute.limit.rejected.handler"`
+	Auth                        string            `yaml:"auth" json:"auth,omitempty" property:"auth"`
+	NotRegister                 bool              `yaml:"not_register" json:"not_register,omitempty" property:"not_register"`
+	ParamSign                   string            `yaml:"param.sign" json:"param.sign,omitempty" property:"param.sign"`
+	Tag                         string            `yaml:"tag" json:"tag,omitempty" property:"tag"`
+	TracingKey                  string            `yaml:"tracing-key" json:"tracing-key,omitempty" propertiy:"tracing-key"`
+
+	RCProtocolsMap  map[string]*ProtocolConfig
+	RCRegistriesMap map[string]*RegistryConfig
+	ProxyFactoryKey string
+}
+
+type ServiceOption func(*ServiceConfig)
+
+func DefaultServiceConfig() *ServiceConfig {
+	return &ServiceConfig{
+		Methods:         make([]*MethodConfig, 0, 8),
+		Params:          make(map[string]string, 8),
+		RCProtocolsMap:  make(map[string]*ProtocolConfig),
+		RCRegistriesMap: make(map[string]*RegistryConfig),
+	}
+}
+
+func WithService_Filter(filter string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Filter = filter
+	}
+}
+
+func WithService_ProtocolIDs(protocolIDs []string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		if len(protocolIDs) <= 0 {
+			cfg.ProtocolIDs = protocolIDs
+		}
+	}
+}
+
+func WithService_Interface(name string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Interface = name
+	}
+}
+
+func WithService_RegistryIDs(registryIDs []string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		if len(registryIDs) <= 0 {
+			cfg.RegistryIDs = registryIDs
+		}
+	}
+}
+
+func WithService_Cluster(cluster string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Cluster = cluster
+	}
+}
+
+func WithService_LoadBalance(loadBalance string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Loadbalance = loadBalance
+	}
+}
+
+func WithService_Group(group string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Group = group
+	}
+}
+
+func WithService_Version(version string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Version = version
+	}
+}
+
+func WithService_Methods(methods []*MethodConfig) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Methods = methods
+	}
+}
+
+func WithService_WarmUp(warmUp string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Warmup = warmUp
+	}
+}
+
+func WithService_Retries(retries string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Retries = retries
+	}
+}
+
+func WithService_Serialization(serialization string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Serialization = serialization
+	}
+}
+
+func WithService_Params(params map[string]string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Params = params
+	}
+}
+
+func WithService_Token(token string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Token = token
+	}
+}
+
+func WithService_AccessLog(accessLog string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.AccessLog = accessLog
+	}
+}
+
+func WithService_TpsLimiter(tpsLimiter string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TpsLimiter = tpsLimiter
+	}
+}
+
+func WithService_TpsLimitInterval(tpsLimitInterval string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TpsLimitInterval = tpsLimitInterval
+	}
+}
+
+func WithService_TpsLimitRate(tpsLimitRate string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TpsLimitRate = tpsLimitRate
+	}
+}
+
+func WithService_TpsLimitStrategy(tpsLimitStrategy string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TpsLimitStrategy = tpsLimitStrategy
+	}
+}
+
+func WithService_TpsLimitRejectedHandler(tpsLimitRejectedHandler string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TpsLimitRejectedHandler = tpsLimitRejectedHandler
+	}
+}
+
+func WithService_ExecuteLimit(executeLimit string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.ExecuteLimit = executeLimit
+	}
+}
+
+func WithService_ExecuteLimitRejectedHandler(executeLimitRejectedHandler string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.ExecuteLimitRejectedHandler = executeLimitRejectedHandler
+	}
+}
+
+func WithService_Auth(auth string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Auth = auth
+	}
+}
+
+func WithService_NotRegister(notRegister bool) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.NotRegister = notRegister
+	}
+}
+
+func WithService_ParamSign(paramSign string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.ParamSign = paramSign
+	}
+}
+
+func WithService_Tag(tag string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.Tag = tag
+	}
+}
+
+func WithService_TracingKey(tracingKey string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.TracingKey = tracingKey
+	}
+}
+
+func WithService_RCProtocol(name string, protocol *ProtocolConfig) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		if cfg.RCProtocolsMap == nil {
+			cfg.RCProtocolsMap = make(map[string]*ProtocolConfig)
+		}
+		cfg.RCProtocolsMap[name] = protocol
+	}
+}
+
+func WithService_RCRegistry(name string, registry *RegistryConfig) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		if cfg.RCRegistriesMap == nil {
+			cfg.RCRegistriesMap = make(map[string]*RegistryConfig)
+		}
+		cfg.RCRegistriesMap[name] = registry
+	}
+}
+
+func WithService_ProxyFactoryKey(factory string) ServiceOption {
+	return func(cfg *ServiceConfig) {
+		cfg.ProxyFactoryKey = factory
+	}
+}
diff --git a/global/shutdown_config.go b/global/shutdown_config.go
new file mode 100644
index 0000000..60dcc1a
--- /dev/null
+++ b/global/shutdown_config.go
@@ -0,0 +1,114 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+import (
+	"github.com/creasty/defaults"
+
+	"go.uber.org/atomic"
+)
+
+// ShutdownConfig is used as configuration for graceful shutdown
+type ShutdownConfig struct {
+	/*
+	 * Total timeout. Even though we don't release all resources,
+	 * the applicationConfig will shutdown if the costing time is over this configuration. The unit is ms.
+	 * default value is 60 * 1000 ms = 1 minutes
+	 * In general, it should be bigger than 3 * StepTimeout.
+	 */
+	Timeout string `default:"60s" yaml:"timeout" json:"timeout,omitempty" property:"timeout"`
+	/*
+	 * the timeout on each step. You should evaluate the response time of request
+	 * and the time that client noticed that server shutdown.
+	 * For example, if your client will received the notification within 10s when you start to close server,
+	 * and the 99.9% requests will return response in 2s, so the StepTimeout will be bigger than(10+2) * 1000ms,
+	 * maybe (10 + 2*3) * 1000ms is a good choice.
+	 */
+	StepTimeout string `default:"3s" yaml:"step-timeout" json:"step.timeout,omitempty" property:"step.timeout"`
+
+	/*
+	 * ConsumerUpdateWaitTime means when provider is shutting down, after the unregister, time to wait for client to
+	 * update invokers. During this time, incoming invocation can be treated normally.
+	 */
+	ConsumerUpdateWaitTime string `default:"3s" yaml:"consumer-update-wait-time" json:"consumerUpdate.waitTIme,omitempty" property:"consumerUpdate.waitTIme"`
+	// when we try to shutdown the applicationConfig, we will reject the new requests. In most cases, you don't need to configure this.
+	RejectRequestHandler string `yaml:"reject-handler" json:"reject-handler,omitempty" property:"reject_handler"`
+	// internal listen kill signal,the default is true.
+	InternalSignal *bool `default:"true" yaml:"internal-signal" json:"internal.signal,omitempty" property:"internal.signal"`
+	// offline request window length
+	OfflineRequestWindowTimeout string `yaml:"offline-request-window-timeout" json:"offlineRequestWindowTimeout,omitempty" property:"offlineRequestWindowTimeout"`
+	// true -> new request will be rejected.
+	RejectRequest atomic.Bool
+	// active invocation
+	ConsumerActiveCount atomic.Int32
+	ProviderActiveCount atomic.Int32
+
+	// provider last received request timestamp
+	ProviderLastReceivedRequestTime atomic.Time
+}
+
+func DefaultShutdownConfig() *ShutdownConfig {
+	cfg := &ShutdownConfig{}
+	defaults.MustSet(cfg)
+
+	return cfg
+}
+
+type ShutdownOption func(*ShutdownConfig)
+
+func WithShutdown_Timeout(timeout string) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.Timeout = timeout
+	}
+}
+
+func WithShutdown_StepTimeout(timeout string) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.StepTimeout = timeout
+	}
+}
+
+func WithShutdown_ConsumerUpdateWaitTime(duration string) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.ConsumerUpdateWaitTime = duration
+	}
+}
+
+func WithShutdown_RejectRequestHandler(handler string) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.RejectRequestHandler = handler
+	}
+}
+
+func WithShutdown_InternalSignal(signal bool) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.InternalSignal = &signal
+	}
+}
+
+func WithShutdown_OfflineRequestWindowTimeout(timeout string) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.OfflineRequestWindowTimeout = timeout
+	}
+}
+
+func WithShutdown_RejectRequest(flag bool) ShutdownOption {
+	return func(cfg *ShutdownConfig) {
+		cfg.RejectRequest.Store(flag)
+	}
+}
diff --git a/global/tls_config.go b/global/tls_config.go
new file mode 100644
index 0000000..f6b1e56
--- /dev/null
+++ b/global/tls_config.go
@@ -0,0 +1,57 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// TLSConfig tls config
+type TLSConfig struct {
+	CACertFile    string `yaml:"ca-cert-file" json:"ca-cert-file" property:"ca-cert-file"`
+	TLSCertFile   string `yaml:"tls-cert-file" json:"tls-cert-file" property:"tls-cert-file"`
+	TLSKeyFile    string `yaml:"tls-key-file" json:"tls-key-file" property:"tls-key-file"`
+	TLSServerName string `yaml:"tls-server-name" json:"tls-server-name" property:"tls-server-name"`
+}
+
+func DefaultTLSConfig() *TLSConfig {
+	// please refer to /config/tls_config.go NewTLSConfigBuilder
+	return nil
+}
+
+type TLSOption func(*TLSConfig)
+
+func WithTLS_CACertFile(file string) TLSOption {
+	return func(cfg *TLSConfig) {
+		cfg.CACertFile = file
+	}
+}
+
+func WithTLS_TLSCertFile(file string) TLSOption {
+	return func(cfg *TLSConfig) {
+		cfg.TLSCertFile = file
+	}
+}
+
+func WithTLS_TLSKeyFile(file string) TLSOption {
+	return func(cfg *TLSConfig) {
+		cfg.TLSKeyFile = file
+	}
+}
+
+func WithTLS_TLSServerName(name string) TLSOption {
+	return func(cfg *TLSConfig) {
+		cfg.TLSServerName = name
+	}
+}
diff --git a/global/tracing_config.go b/global/tracing_config.go
new file mode 100644
index 0000000..d6d453f
--- /dev/null
+++ b/global/tracing_config.go
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package global
+
+// TracingConfig is the configuration of the tracing.
+type TracingConfig struct {
+	Name        string `default:"jaeger" yaml:"name" json:"name,omitempty" property:"name"` // jaeger or zipkin(todo)
+	ServiceName string `yaml:"serviceName" json:"serviceName,omitempty" property:"serviceName"`
+	Address     string `yaml:"address" json:"address,omitempty" property:"address"`
+	UseAgent    *bool  `default:"false" yaml:"use-agent" json:"use-agent,omitempty" property:"use-agent"`
+}
+
+type TracingOption func(*TracingConfig)
+
+func WithTracing_Name(name string) TracingOption {
+	return func(cfg *TracingConfig) {
+		cfg.Name = name
+	}
+}
+
+func WithTracing_ServiceName(name string) TracingOption {
+	return func(cfg *TracingConfig) {
+		cfg.ServiceName = name
+	}
+}
+
+func WithTracing_Address(address string) TracingOption {
+	return func(cfg *TracingConfig) {
+		cfg.Address = address
+	}
+}
+
+func WithTracing_UseAgent(flag bool) TracingOption {
+	return func(cfg *TracingConfig) {
+		cfg.UseAgent = &flag
+	}
+}
diff --git a/graceful_shutdown/common.go b/graceful_shutdown/common.go
new file mode 100644
index 0000000..82cc4cc
--- /dev/null
+++ b/graceful_shutdown/common.go
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"time"
+)
+
+import (
+	hessian "github.com/apache/dubbo-go-hessian2"
+
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+)
+
+func init() {
+	hessian.RegisterPOJO(&common.MetadataInfo{})
+	hessian.RegisterPOJO(&common.ServiceInfo{})
+	hessian.RegisterPOJO(&common.URL{})
+}
+
+func parseDuration(timeout string, desc string, def time.Duration) time.Duration {
+	res, err := time.ParseDuration(timeout)
+	if err != nil {
+		logger.Errorf("The %s configuration is invalid: %s, and we will use the default value: %s, err: %v",
+			desc, timeout, def.String(), err)
+		res = def
+	}
+
+	return res
+}
diff --git a/graceful_shutdown/compat.go b/graceful_shutdown/compat.go
new file mode 100644
index 0000000..f8b80bb
--- /dev/null
+++ b/graceful_shutdown/compat.go
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"go.uber.org/atomic"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+func compatShutdownConfig(c *global.ShutdownConfig) *config.ShutdownConfig {
+	if c == nil {
+		return nil
+	}
+	cfg := &config.ShutdownConfig{
+		Timeout:                     c.Timeout,
+		StepTimeout:                 c.StepTimeout,
+		ConsumerUpdateWaitTime:      c.ConsumerUpdateWaitTime,
+		RejectRequestHandler:        c.RejectRequestHandler,
+		InternalSignal:              c.InternalSignal,
+		OfflineRequestWindowTimeout: c.OfflineRequestWindowTimeout,
+		RejectRequest:               atomic.Bool{},
+	}
+	cfg.RejectRequest.Store(c.RejectRequest.Load())
+	return cfg
+}
diff --git a/graceful_shutdown/graceful_shutdown_signal_darwin.go b/graceful_shutdown/graceful_shutdown_signal_darwin.go
new file mode 100644
index 0000000..e9619f0
--- /dev/null
+++ b/graceful_shutdown/graceful_shutdown_signal_darwin.go
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"os"
+	"syscall"
+)
+
+var (
+	// ShutdownSignals receives shutdown signals to process
+	ShutdownSignals = []os.Signal{
+		os.Interrupt, os.Kill, syscall.SIGKILL, syscall.SIGSTOP,
+		syscall.SIGHUP, syscall.SIGINT, syscall.SIGQUIT, syscall.SIGILL, syscall.SIGTRAP,
+		syscall.SIGABRT, syscall.SIGSYS, syscall.SIGTERM,
+	}
+
+	// DumpHeapShutdownSignals receives shutdown signals to process
+	DumpHeapShutdownSignals = []os.Signal{
+		syscall.SIGQUIT, syscall.SIGILL,
+		syscall.SIGTRAP, syscall.SIGABRT, syscall.SIGSYS,
+	}
+)
diff --git a/graceful_shutdown/graceful_shutdown_signal_linux.go b/graceful_shutdown/graceful_shutdown_signal_linux.go
new file mode 100644
index 0000000..e9619f0
--- /dev/null
+++ b/graceful_shutdown/graceful_shutdown_signal_linux.go
@@ -0,0 +1,38 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"os"
+	"syscall"
+)
+
+var (
+	// ShutdownSignals receives shutdown signals to process
+	ShutdownSignals = []os.Signal{
+		os.Interrupt, os.Kill, syscall.SIGKILL, syscall.SIGSTOP,
+		syscall.SIGHUP, syscall.SIGINT, syscall.SIGQUIT, syscall.SIGILL, syscall.SIGTRAP,
+		syscall.SIGABRT, syscall.SIGSYS, syscall.SIGTERM,
+	}
+
+	// DumpHeapShutdownSignals receives shutdown signals to process
+	DumpHeapShutdownSignals = []os.Signal{
+		syscall.SIGQUIT, syscall.SIGILL,
+		syscall.SIGTRAP, syscall.SIGABRT, syscall.SIGSYS,
+	}
+)
diff --git a/graceful_shutdown/graceful_shutdown_signal_windows.go b/graceful_shutdown/graceful_shutdown_signal_windows.go
new file mode 100644
index 0000000..ad21acd
--- /dev/null
+++ b/graceful_shutdown/graceful_shutdown_signal_windows.go
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"os"
+	"syscall"
+)
+
+var (
+	// ShutdownSignals receives shutdown signals to process
+	ShutdownSignals = []os.Signal{
+		os.Interrupt, os.Kill, syscall.SIGKILL,
+		syscall.SIGHUP, syscall.SIGINT, syscall.SIGQUIT, syscall.SIGILL, syscall.SIGTRAP,
+		syscall.SIGABRT, syscall.SIGTERM,
+	}
+
+	// DumpHeapShutdownSignals receives shutdown signals to process
+	DumpHeapShutdownSignals = []os.Signal{syscall.SIGQUIT, syscall.SIGILL, syscall.SIGTRAP, syscall.SIGABRT}
+)
diff --git a/graceful_shutdown/options.go b/graceful_shutdown/options.go
new file mode 100644
index 0000000..e364bb9
--- /dev/null
+++ b/graceful_shutdown/options.go
@@ -0,0 +1,94 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"time"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+var (
+	defOpts = &Options{
+		Shutdown: global.DefaultShutdownConfig(),
+	}
+)
+
+type Options struct {
+	Shutdown *global.ShutdownConfig
+}
+
+func DefaultOptions() *Options {
+	return defOpts
+}
+
+type Option func(*Options)
+
+func WithTimeout(timeout time.Duration) Option {
+	return func(opts *Options) {
+		opts.Shutdown.Timeout = timeout.String()
+	}
+}
+
+func WithStepTimeout(timeout time.Duration) Option {
+	return func(opts *Options) {
+		opts.Shutdown.StepTimeout = timeout.String()
+	}
+}
+
+func WithConsumerUpdateWaitTime(duration time.Duration) Option {
+	return func(opts *Options) {
+		opts.Shutdown.ConsumerUpdateWaitTime = duration.String()
+	}
+}
+
+// todo(DMwangnima): add more specified configuration API
+//func WithRejectRequestHandler(handler string) Option {
+//	return func(opts *Options) {
+//		opts.Shutdown.RejectRequestHandler = handler
+//	}
+//}
+
+func WithoutInternalSignal() Option {
+	return func(opts *Options) {
+		signal := false
+		opts.Shutdown.InternalSignal = &signal
+	}
+}
+
+func WithOfflineRequestWindowTimeout(timeout time.Duration) Option {
+	return func(opts *Options) {
+		opts.Shutdown.OfflineRequestWindowTimeout = timeout.String()
+	}
+}
+
+func WithRejectRequest() Option {
+	return func(opts *Options) {
+		opts.Shutdown.RejectRequest.Store(true)
+	}
+}
+
+// ---------- For framework ----------
+
+func WithShutdown_Config(cfg *global.ShutdownConfig) Option {
+	return func(opts *Options) {
+		opts.Shutdown = cfg
+	}
+}
diff --git a/graceful_shutdown/shutdown.go b/graceful_shutdown/shutdown.go
new file mode 100644
index 0000000..b560ed6
--- /dev/null
+++ b/graceful_shutdown/shutdown.go
@@ -0,0 +1,216 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package graceful_shutdown
+
+import (
+	"os"
+	"os/signal"
+	"runtime/debug"
+	"sync"
+	"time"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/config"
+)
+
+const (
+	// todo(DMwangnima): these descriptions and defaults could be wrapped by functions of Options
+	defaultTimeout                     = 60 * time.Second
+	defaultStepTimeout                 = 3 * time.Second
+	defaultConsumerUpdateWaitTime      = 3 * time.Second
+	defaultOfflineRequestWindowTimeout = 3 * time.Second
+
+	timeoutDesc                     = "Timeout"
+	stepTimeoutDesc                 = "StepTimeout"
+	consumerUpdateWaitTimeDesc      = "ConsumerUpdateWaitTime"
+	offlineRequestWindowTimeoutDesc = "OfflineRequestWindowTimeout"
+)
+
+var (
+	initOnce       sync.Once
+	compatShutdown *config.ShutdownConfig
+
+	proMu     sync.Mutex
+	protocols map[string]struct{}
+)
+
+func Init(opts ...Option) {
+	initOnce.Do(func() {
+		protocols = make(map[string]struct{})
+		newOpts := DefaultOptions()
+		for _, opt := range opts {
+			opt(newOpts)
+		}
+		compatShutdown = compatShutdownConfig(newOpts.Shutdown)
+		// retrieve ShutdownConfig for gracefulShutdownFilter
+		cGracefulShutdownFilter, existcGracefulShutdownFilter := extension.GetFilter(constant.GracefulShutdownConsumerFilterKey)
+		if !existcGracefulShutdownFilter {
+			return
+		}
+		sGracefulShutdownFilter, existsGracefulShutdownFilter := extension.GetFilter(constant.GracefulShutdownProviderFilterKey)
+		if !existsGracefulShutdownFilter {
+			return
+		}
+		if filter, ok := cGracefulShutdownFilter.(config.Setter); ok {
+			filter.Set(constant.GracefulShutdownFilterShutdownConfig, compatShutdown)
+		}
+
+		if filter, ok := sGracefulShutdownFilter.(config.Setter); ok {
+			filter.Set(constant.GracefulShutdownFilterShutdownConfig, compatShutdown)
+		}
+
+		if compatShutdown.InternalSignal != nil && *compatShutdown.InternalSignal {
+			signals := make(chan os.Signal, 1)
+			signal.Notify(signals, ShutdownSignals...)
+
+			go func() {
+				select {
+				case sig := <-signals:
+					logger.Infof("get signal %s, applicationConfig will shutdown.", sig)
+					// gracefulShutdownOnce.Do(func() {
+					time.AfterFunc(totalTimeout(), func() {
+						logger.Warn("Shutdown gracefully timeout, applicationConfig will shutdown immediately. ")
+						os.Exit(0)
+					})
+					beforeShutdown()
+					// those signals' original behavior is exit with dump ths stack, so we try to keep the behavior
+					for _, dumpSignal := range DumpHeapShutdownSignals {
+						if sig == dumpSignal {
+							debug.WriteHeapDump(os.Stdout.Fd())
+						}
+					}
+					os.Exit(0)
+				}
+			}()
+		}
+	})
+}
+
+// RegisterProtocol registers protocol which would be destroyed before shutdown.
+// Please make sure that Init function has been invoked before, otherwise this
+// function would not make any sense.
+func RegisterProtocol(name string) {
+	proMu.Lock()
+	protocols[name] = struct{}{}
+	proMu.Unlock()
+}
+
+func totalTimeout() time.Duration {
+	timeout := parseDuration(compatShutdown.Timeout, timeoutDesc, defaultTimeout)
+	if timeout < defaultTimeout {
+		timeout = defaultTimeout
+	}
+
+	return timeout
+}
+
+func beforeShutdown() {
+	destroyRegistries()
+	// waiting for a short time so that the clients have enough time to get the notification that server shutdowns
+	// The value of configuration depends on how long the clients will get notification.
+	waitAndAcceptNewRequests()
+
+	// reject sending/receiving the new request, but keeping waiting for accepting requests
+	waitForSendingAndReceivingRequests()
+
+	// destroy all protocols
+	destroyProtocols()
+
+	logger.Info("Graceful shutdown --- Execute the custom callbacks.")
+	customCallbacks := extension.GetAllCustomShutdownCallbacks()
+	for callback := customCallbacks.Front(); callback != nil; callback = callback.Next() {
+		callback.Value.(func())()
+	}
+}
+
+// destroyRegistries destroys RegistryProtocol directly.
+func destroyRegistries() {
+	logger.Info("Graceful shutdown --- Destroy all registriesConfig. ")
+	registryProtocol := extension.GetProtocol(constant.RegistryProtocol)
+	registryProtocol.Destroy()
+}
+
+func waitAndAcceptNewRequests() {
+	logger.Info("Graceful shutdown --- Keep waiting and accept new requests for a short time. ")
+
+	updateWaitTime := parseDuration(compatShutdown.ConsumerUpdateWaitTime, consumerUpdateWaitTimeDesc, defaultConsumerUpdateWaitTime)
+	time.Sleep(updateWaitTime)
+
+	stepTimeout := parseDuration(compatShutdown.StepTimeout, stepTimeoutDesc, defaultStepTimeout)
+
+	// ignore this step
+	if stepTimeout < 0 {
+		return
+	}
+	waitingProviderProcessedTimeout(stepTimeout)
+}
+
+func waitingProviderProcessedTimeout(timeout time.Duration) {
+	deadline := time.Now().Add(timeout)
+
+	offlineRequestWindowTimeout := parseDuration(compatShutdown.OfflineRequestWindowTimeout, offlineRequestWindowTimeoutDesc, defaultOfflineRequestWindowTimeout)
+
+	for time.Now().Before(deadline) &&
+		(compatShutdown.ProviderActiveCount.Load() > 0 || time.Now().Before(compatShutdown.ProviderLastReceivedRequestTime.Load().Add(offlineRequestWindowTimeout))) {
+		// sleep 10 ms and then we check it again
+		time.Sleep(10 * time.Millisecond)
+		logger.Infof("waiting for provider active invocation count = %d, provider last received request time: %v",
+			compatShutdown.ProviderActiveCount.Load(), compatShutdown.ProviderLastReceivedRequestTime.Load())
+	}
+}
+
+// for provider. It will wait for processing receiving requests
+func waitForSendingAndReceivingRequests() {
+	logger.Info("Graceful shutdown --- Keep waiting until sending/accepting requests finish or timeout. ")
+	compatShutdown.RejectRequest.Store(true)
+	waitingConsumerProcessedTimeout()
+}
+
+func waitingConsumerProcessedTimeout() {
+	stepTimeout := parseDuration(compatShutdown.StepTimeout, stepTimeoutDesc, defaultStepTimeout)
+
+	if stepTimeout <= 0 {
+		return
+	}
+	deadline := time.Now().Add(stepTimeout)
+
+	for time.Now().Before(deadline) && compatShutdown.ConsumerActiveCount.Load() > 0 {
+		// sleep 10 ms and then we check it again
+		time.Sleep(10 * time.Millisecond)
+		logger.Infof("waiting for consumer active invocation count = %d", compatShutdown.ConsumerActiveCount.Load())
+	}
+}
+
+// destroyProtocols destroys protocols that have been registered.
+func destroyProtocols() {
+	logger.Info("Graceful shutdown --- Destroy protocols. ")
+
+	proMu.Lock()
+	// extension.GetProtocol might panic
+	defer proMu.Unlock()
+	for name := range protocols {
+		extension.GetProtocol(name).Destroy()
+	}
+}
diff --git a/imports/imports.go b/imports/imports.go
index 90737e9..3bc668a 100644
--- a/imports/imports.go
+++ b/imports/imports.go
@@ -77,6 +77,7 @@
 	_ "dubbo.apache.org/dubbo-go/v3/protocol/grpc"
 	_ "dubbo.apache.org/dubbo-go/v3/protocol/jsonrpc"
 	_ "dubbo.apache.org/dubbo-go/v3/protocol/rest"
+	_ "dubbo.apache.org/dubbo-go/v3/protocol/triple"
 	_ "dubbo.apache.org/dubbo-go/v3/proxy/proxy_factory"
 	_ "dubbo.apache.org/dubbo-go/v3/registry/etcdv3"
 	_ "dubbo.apache.org/dubbo-go/v3/registry/nacos"
diff --git a/loader.go b/loader.go
new file mode 100644
index 0000000..5679c25
--- /dev/null
+++ b/loader.go
@@ -0,0 +1,347 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package dubbo
+
+import (
+	"io/ioutil"
+	"os"
+	"path/filepath"
+	"runtime"
+	"strings"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+
+	"github.com/knadh/koanf"
+	"github.com/knadh/koanf/parsers/json"
+	"github.com/knadh/koanf/parsers/toml"
+	"github.com/knadh/koanf/parsers/yaml"
+	"github.com/knadh/koanf/providers/confmap"
+	"github.com/knadh/koanf/providers/rawbytes"
+
+	"github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/constant/file"
+	"dubbo.apache.org/dubbo-go/v3/config/parsers/properties"
+)
+
+var (
+	defaultActive   = "default"
+	instanceOptions = defaultInstanceOptions()
+)
+
+func Load(opts ...LoaderConfOption) error {
+	// conf
+	conf := NewLoaderConf(opts...)
+	if conf.opts == nil {
+		koan := GetConfigResolver(conf)
+		koan = conf.MergeConfig(koan)
+		if err := koan.UnmarshalWithConf(instanceOptions.Prefix(),
+			instanceOptions, koanf.UnmarshalConf{Tag: "yaml"}); err != nil {
+			return err
+		}
+	} else {
+		instanceOptions = conf.opts
+	}
+
+	if err := instanceOptions.init(); err != nil {
+		return err
+	}
+
+	// todo(DMwangnima): use independent Consumer and Provider logic
+	return nil
+}
+
+type loaderConf struct {
+	suffix string           // loaderConf file extension default yaml
+	path   string           // loaderConf file path default ./conf/dubbogo.yaml
+	delim  string           // loaderConf file delim default .
+	bytes  []byte           // config bytes
+	opts   *InstanceOptions // user provide InstanceOptions built by WithXXX api
+	name   string           // config file name
+}
+
+func NewLoaderConf(opts ...LoaderConfOption) *loaderConf {
+	configFilePath := "../conf/dubbogo.yaml"
+	if configFilePathFromEnv := os.Getenv(constant.ConfigFileEnvKey); configFilePathFromEnv != "" {
+		configFilePath = configFilePathFromEnv
+	}
+	name, suffix := resolverFilePath(configFilePath)
+	conf := &loaderConf{
+		suffix: suffix,
+		path:   absolutePath(configFilePath),
+		delim:  ".",
+		name:   name,
+	}
+	for _, opt := range opts {
+		opt.apply(conf)
+	}
+	if conf.opts != nil {
+		return conf
+	}
+	if len(conf.bytes) <= 0 {
+		if bytes, err := ioutil.ReadFile(conf.path); err != nil {
+			panic(err)
+		} else {
+			conf.bytes = bytes
+		}
+	}
+	return conf
+}
+
+type LoaderConfOption interface {
+	apply(vc *loaderConf)
+}
+
+type loaderConfigFunc func(*loaderConf)
+
+func (fn loaderConfigFunc) apply(vc *loaderConf) {
+	fn(vc)
+}
+
+// WithGenre set load config file suffix
+// Deprecated: replaced by WithSuffix
+func WithGenre(suffix string) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		g := strings.ToLower(suffix)
+		if err := checkFileSuffix(g); err != nil {
+			panic(err)
+		}
+		conf.suffix = g
+	})
+}
+
+// WithSuffix set load config file suffix
+func WithSuffix(suffix file.Suffix) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		conf.suffix = string(suffix)
+	})
+}
+
+// WithPath set load config path
+func WithPath(path string) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		conf.path = absolutePath(path)
+		if bytes, err := ioutil.ReadFile(conf.path); err != nil {
+			panic(err)
+		} else {
+			conf.bytes = bytes
+		}
+		name, suffix := resolverFilePath(path)
+		conf.suffix = suffix
+		conf.name = name
+	})
+}
+
+func WithInstanceOptions(opts *InstanceOptions) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		conf.opts = opts
+	})
+}
+
+func WithDelim(delim string) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		conf.delim = delim
+	})
+}
+
+// WithBytes set load config  bytes
+func WithBytes(bytes []byte) LoaderConfOption {
+	return loaderConfigFunc(func(conf *loaderConf) {
+		conf.bytes = bytes
+	})
+}
+
+// absolutePath get absolut path
+func absolutePath(inPath string) string {
+
+	if inPath == "$HOME" || strings.HasPrefix(inPath, "$HOME"+string(os.PathSeparator)) {
+		inPath = userHomeDir() + inPath[5:]
+	}
+
+	if filepath.IsAbs(inPath) {
+		return filepath.Clean(inPath)
+	}
+
+	p, err := filepath.Abs(inPath)
+	if err == nil {
+		return filepath.Clean(p)
+	}
+
+	return ""
+}
+
+// userHomeDir get gopath
+func userHomeDir() string {
+	if runtime.GOOS == "windows" {
+		home := os.Getenv("HOMEDRIVE") + os.Getenv("HOMEPATH")
+		if home == "" {
+			home = os.Getenv("USERPROFILE")
+		}
+		return home
+	}
+	return os.Getenv("HOME")
+}
+
+// checkFileSuffix check file suffix
+func checkFileSuffix(suffix string) error {
+	for _, g := range []string{"json", "toml", "yaml", "yml", "properties"} {
+		if g == suffix {
+			return nil
+		}
+	}
+	return errors.Errorf("no support file suffix: %s", suffix)
+}
+
+// resolverFilePath resolver file path
+// eg: give a ./conf/dubbogo.yaml return dubbogo and yaml
+func resolverFilePath(path string) (name, suffix string) {
+	paths := strings.Split(path, "/")
+	fileName := strings.Split(paths[len(paths)-1], ".")
+	if len(fileName) < 2 {
+		return fileName[0], string(file.YAML)
+	}
+	return fileName[0], fileName[1]
+}
+
+// MergeConfig merge config file
+func (conf *loaderConf) MergeConfig(koan *koanf.Koanf) *koanf.Koanf {
+	var (
+		activeKoan *koanf.Koanf
+		activeConf *loaderConf
+	)
+	active := koan.String("dubbo.profiles.active")
+	active = getLegalActive(active)
+	logger.Infof("The following profiles are active: %s", active)
+	if defaultActive != active {
+		path := conf.getActiveFilePath(active)
+		if !pathExists(path) {
+			logger.Debugf("Config file:%s not exist skip config merge", path)
+			return koan
+		}
+		activeConf = NewLoaderConf(WithPath(path))
+		activeKoan = GetConfigResolver(activeConf)
+		if err := koan.Merge(activeKoan); err != nil {
+			logger.Debugf("Config merge err %s", err)
+		}
+	}
+	return koan
+}
+
+func (conf *loaderConf) getActiveFilePath(active string) string {
+	suffix := constant.DotSeparator + conf.suffix
+	return strings.ReplaceAll(conf.path, suffix, "") + "-" + active + suffix
+}
+
+func pathExists(path string) bool {
+	if _, err := os.Stat(path); err == nil {
+		return true
+	} else {
+		return !os.IsNotExist(err)
+	}
+}
+
+// getLegalActive if active is null return default
+func getLegalActive(active string) string {
+	if len(active) == 0 {
+		return defaultActive
+	}
+	return active
+}
+
+// GetConfigResolver get config resolver
+func GetConfigResolver(conf *loaderConf) *koanf.Koanf {
+	var (
+		k   *koanf.Koanf
+		err error
+	)
+	if len(conf.suffix) <= 0 {
+		conf.suffix = string(file.YAML)
+	}
+	if len(conf.delim) <= 0 {
+		conf.delim = "."
+	}
+	bytes := conf.bytes
+	if len(bytes) <= 0 {
+		panic(errors.New("bytes is nil,please set bytes or file path"))
+	}
+	k = koanf.New(conf.delim)
+
+	switch conf.suffix {
+	case "yaml", "yml":
+		err = k.Load(rawbytes.Provider(bytes), yaml.Parser())
+	case "json":
+		err = k.Load(rawbytes.Provider(bytes), json.Parser())
+	case "toml":
+		err = k.Load(rawbytes.Provider(bytes), toml.Parser())
+	case "properties":
+		err = k.Load(rawbytes.Provider(bytes), properties.Parser())
+	default:
+		err = errors.Errorf("no support %s file suffix", conf.suffix)
+	}
+
+	if err != nil {
+		panic(err)
+	}
+	return resolvePlaceholder(k)
+}
+
+// resolvePlaceholder replace ${xx} with real value
+func resolvePlaceholder(resolver *koanf.Koanf) *koanf.Koanf {
+	m := make(map[string]interface{})
+	for k, v := range resolver.All() {
+		s, ok := v.(string)
+		if !ok {
+			continue
+		}
+		newKey, defaultValue := checkPlaceholder(s)
+		if newKey == "" {
+			continue
+		}
+		m[k] = resolver.Get(newKey)
+		if m[k] == nil {
+			m[k] = defaultValue
+		}
+	}
+	err := resolver.Load(confmap.Provider(m, resolver.Delim()), nil)
+	if err != nil {
+		logger.Errorf("resolvePlaceholder error %s", err)
+	}
+	return resolver
+}
+
+func checkPlaceholder(s string) (newKey, defaultValue string) {
+	s = strings.TrimSpace(s)
+	if !strings.HasPrefix(s, file.PlaceholderPrefix) || !strings.HasSuffix(s, file.PlaceholderSuffix) {
+		return
+	}
+	s = s[len(file.PlaceholderPrefix) : len(s)-len(file.PlaceholderSuffix)]
+	indexColon := strings.Index(s, ":")
+	if indexColon == -1 {
+		newKey = strings.TrimSpace(s)
+		return
+	}
+	newKey = strings.TrimSpace(s[0:indexColon])
+	defaultValue = strings.TrimSpace(s[indexColon+1:])
+
+	return
+}
diff --git a/metadata/metadata.go b/metadata/metadata.go
new file mode 100644
index 0000000..b35ac29
--- /dev/null
+++ b/metadata/metadata.go
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package metadata
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+
+	perrors "github.com/pkg/errors"
+
+	"go.uber.org/atomic"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/metadata/service/exporter"
+)
+
+var (
+	exporting = &atomic.Bool{}
+)
+
+func ExportMetadataService() {
+	ms, err := extension.GetLocalMetadataService(constant.DefaultKey)
+	if err != nil {
+		logger.Warnf("could not init metadata service", err)
+		return
+	}
+
+	if exporting.Load() {
+		return
+	}
+
+	// In theory, we can use sync.Once
+	// But sync.Once is not reentrant.
+	// Now the invocation chain is createRegistry -> tryInitMetadataService -> metadataServiceExporter.export
+	// -> createRegistry -> initMetadataService...
+	// So using sync.Once will result in dead lock
+	exporting.Store(true)
+
+	expt := extension.GetMetadataServiceExporter(constant.DefaultKey, ms)
+	if expt == nil {
+		logger.Warnf("get metadata service exporter failed, pls check if you import _ \"dubbo.apache.org/dubbo-go/v3/metadata/service/exporter/configurable\"")
+		return
+	}
+
+	err = expt.Export(nil)
+	if err != nil {
+		logger.Errorf("could not export the metadata service, err = %s", err.Error())
+		return
+	}
+
+	// report interface-app mapping
+	err = publishMapping(expt)
+	if err != nil {
+		logger.Errorf("Publish interface-application mapping failed, got error %#v", err)
+	}
+}
+
+// OnEvent only handle ServiceConfigExportedEvent
+func publishMapping(sc exporter.MetadataServiceExporter) error {
+	urls := sc.GetExportedURLs()
+
+	for _, u := range urls {
+		err := extension.GetGlobalServiceNameMapping().Map(u)
+		if err != nil {
+			return perrors.WithMessage(err, "could not map the service: "+u.String())
+		}
+	}
+	return nil
+}
diff --git a/metadata/service/local/service.go b/metadata/service/local/service.go
index aeac070..440953e 100644
--- a/metadata/service/local/service.go
+++ b/metadata/service/local/service.go
@@ -67,6 +67,7 @@
 func GetLocalMetadataService() (service.MetadataService, error) {
 	metadataServiceInitOnce.Do(func() {
 		metadataServiceInstance = &MetadataService{
+			// todo(DMwangnima): use external config
 			BaseMetadataService:   service.NewBaseMetadataService(config.GetApplicationConfig().Name),
 			exportedServiceURLs:   &sync.Map{},
 			subscribedServiceURLs: &sync.Map{},
diff --git a/options.go b/options.go
new file mode 100644
index 0000000..51e9b2d
--- /dev/null
+++ b/options.go
@@ -0,0 +1,375 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package dubbo
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+	"dubbo.apache.org/dubbo-go/v3/graceful_shutdown"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+type InstanceOptions struct {
+	Application    *global.ApplicationConfig         `validate:"required" yaml:"application" json:"application,omitempty" property:"application"`
+	Protocols      map[string]*global.ProtocolConfig `validate:"required" yaml:"protocols" json:"protocols" property:"protocols"`
+	Registries     map[string]*global.RegistryConfig `yaml:"registries" json:"registries" property:"registries"`
+	ConfigCenter   *global.CenterConfig              `yaml:"config-center" json:"config-center,omitempty"`
+	MetadataReport *global.MetadataReportConfig      `yaml:"metadata-report" json:"metadata-report,omitempty" property:"metadata-report"`
+	Provider       *global.ProviderConfig            `yaml:"provider" json:"provider" property:"provider"`
+	Consumer       *global.ConsumerConfig            `yaml:"consumer" json:"consumer" property:"consumer"`
+	Metric         *global.MetricConfig              `yaml:"metrics" json:"metrics,omitempty" property:"metrics"`
+	Tracing        map[string]*global.TracingConfig  `yaml:"tracing" json:"tracing,omitempty" property:"tracing"`
+	Logger         *global.LoggerConfig              `yaml:"logger" json:"logger,omitempty" property:"logger"`
+	Shutdown       *global.ShutdownConfig            `yaml:"shutdown" json:"shutdown,omitempty" property:"shutdown"`
+	// todo(DMwangnima): router feature would be supported in the future
+	//Router              []*RouterConfig                   `yaml:"router" json:"router,omitempty" property:"router"`
+	EventDispatcherType string                 `default:"direct" yaml:"event-dispatcher-type" json:"event-dispatcher-type,omitempty"`
+	CacheFile           string                 `yaml:"cache_file" json:"cache_file,omitempty" property:"cache_file"`
+	Custom              *global.CustomConfig   `yaml:"custom" json:"custom,omitempty" property:"custom"`
+	Profiles            *global.ProfilesConfig `yaml:"profiles" json:"profiles,omitempty" property:"profiles"`
+	TLSConfig           *global.TLSConfig      `yaml:"tls_config" json:"tls_config,omitempty" property:"tls_config"`
+}
+
+func defaultInstanceOptions() *InstanceOptions {
+	return &InstanceOptions{
+		Application:    global.DefaultApplicationConfig(),
+		Protocols:      make(map[string]*global.ProtocolConfig),
+		Registries:     make(map[string]*global.RegistryConfig),
+		ConfigCenter:   global.DefaultCenterConfig(),
+		MetadataReport: global.DefaultMetadataReportConfig(),
+		Provider:       global.DefaultProviderConfig(),
+		Consumer:       global.DefaultConsumerConfig(),
+		Metric:         global.DefaultMetricConfig(),
+		Tracing:        make(map[string]*global.TracingConfig),
+		Logger:         global.DefaultLoggerConfig(),
+		Shutdown:       global.DefaultShutdownConfig(),
+		Custom:         global.DefaultCustomConfig(),
+		Profiles:       global.DefaultProfilesConfig(),
+		TLSConfig:      global.DefaultTLSConfig(),
+	}
+}
+
+func (rc *InstanceOptions) init(opts ...InstanceOption) error {
+	for _, opt := range opts {
+		opt(rc)
+	}
+
+	// remaining procedure is like RootConfig.Init() without RootConfig.Start()
+	// tasks of RootConfig.Start() would be decomposed to Client and Server
+	rcCompat := compatRootConfig(rc)
+	if err := rcCompat.Logger.Init(); err != nil { // init default logger
+		return err
+	}
+	if err := rcCompat.ConfigCenter.Init(rcCompat); err != nil {
+		logger.Infof("[Config Center] Config center doesn't start")
+		logger.Debugf("config center doesn't start because %s", err)
+	} else {
+		if err = rcCompat.Logger.Init(); err != nil { // init logger using config from config center again
+			return err
+		}
+	}
+
+	if err := rcCompat.Application.Init(); err != nil {
+		return err
+	}
+
+	// init user define
+	if err := rcCompat.Custom.Init(); err != nil {
+		return err
+	}
+
+	// init protocol
+	protocols := rcCompat.Protocols
+	if len(protocols) <= 0 {
+		protocol := &config.ProtocolConfig{}
+		protocols = make(map[string]*config.ProtocolConfig, 1)
+		protocols[constant.Dubbo] = protocol
+		rcCompat.Protocols = protocols
+	}
+	for _, protocol := range protocols {
+		if err := protocol.Init(); err != nil {
+			return err
+		}
+	}
+
+	// init registry
+	registries := rcCompat.Registries
+	if registries != nil {
+		for _, reg := range registries {
+			if err := reg.Init(); err != nil {
+				return err
+			}
+		}
+	}
+
+	if err := rcCompat.MetadataReport.Init(rcCompat); err != nil {
+		return err
+	}
+	if err := rcCompat.Metric.Init(); err != nil {
+		return err
+	}
+	for _, t := range rcCompat.Tracing {
+		if err := t.Init(); err != nil {
+			return err
+		}
+	}
+
+	routers := rcCompat.Router
+	if len(routers) > 0 {
+		for _, r := range routers {
+			if err := r.Init(); err != nil {
+				return err
+			}
+		}
+		rcCompat.Router = routers
+	}
+
+	// provider、consumer must last init
+	if err := rcCompat.Provider.Init(rcCompat); err != nil {
+		return err
+	}
+	if err := rcCompat.Consumer.Init(rcCompat); err != nil {
+		return err
+	}
+	if err := rcCompat.Shutdown.Init(); err != nil {
+		return err
+	}
+	config.SetRootConfig(*rcCompat)
+
+	return nil
+}
+
+func (rc *InstanceOptions) Prefix() string {
+	return constant.Dubbo
+}
+
+type InstanceOption func(*InstanceOptions)
+
+func WithOrganization(organization string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Organization = organization
+	}
+}
+
+func WithName(name string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Name = name
+	}
+}
+
+func WithModule(module string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Module = module
+	}
+}
+
+func WithGroup(group string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Group = group
+	}
+}
+
+func WithVersion(version string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Version = version
+	}
+}
+
+func WithOwner(owner string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Owner = owner
+	}
+}
+
+func WithEnvironment(environment string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Environment = environment
+	}
+}
+
+func WithRemoteMetadata() InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.MetadataType = constant.RemoteMetadataStorageType
+	}
+}
+
+func WithTag(tag string) InstanceOption {
+	return func(opts *InstanceOptions) {
+		opts.Application.Tag = tag
+	}
+}
+
+func WithProtocol(key string, opts ...protocol.Option) InstanceOption {
+	proOpts := protocol.DefaultOptions()
+	for _, opt := range opts {
+		opt(proOpts)
+	}
+
+	return func(insOpts *InstanceOptions) {
+		if insOpts.Protocols == nil {
+			insOpts.Protocols = make(map[string]*global.ProtocolConfig)
+		}
+		insOpts.Protocols[key] = proOpts.Protocol
+	}
+}
+
+func WithRegistry(key string, opts ...registry.Option) InstanceOption {
+	regOpts := registry.DefaultOptions()
+	for _, opt := range opts {
+		opt(regOpts)
+	}
+
+	return func(insOpts *InstanceOptions) {
+		if insOpts.Registries == nil {
+			insOpts.Registries = make(map[string]*global.RegistryConfig)
+		}
+		insOpts.Registries[key] = regOpts.Registry
+	}
+}
+
+//func WithConfigCenter(opts ...global.CenterOption) InstanceOption {
+//	ccCfg := new(global.CenterConfig)
+//	for _, opt := range opts {
+//		opt(ccCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.ConfigCenter = ccCfg
+//	}
+//}
+
+//func WithMetadataReport(opts ...global.MetadataReportOption) InstanceOption {
+//	mrCfg := new(global.MetadataReportConfig)
+//	for _, opt := range opts {
+//		opt(mrCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.MetadataReport = mrCfg
+//	}
+//}
+
+//func WithConsumer(opts ...global.ConsumerOption) InstanceOption {
+//	conCfg := new(global.ConsumerConfig)
+//	for _, opt := range opts {
+//		opt(conCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.Consumer = conCfg
+//	}
+//}
+
+//func WithMetric(opts ...global.MetricOption) InstanceOption {
+//	meCfg := new(global.MetricConfig)
+//	for _, opt := range opts {
+//		opt(meCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.Metric = meCfg
+//	}
+//}
+
+//func WithTracing(key string, opts ...global.TracingOption) InstanceOption {
+//	traCfg := new(global.TracingConfig)
+//	for _, opt := range opts {
+//		opt(traCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		if cfg.Tracing == nil {
+//			cfg.Tracing = make(map[string]*global.TracingConfig)
+//		}
+//		cfg.Tracing[key] = traCfg
+//	}
+//}
+
+//func WithLogger(opts ...global.LoggerOption) InstanceOption {
+//	logCfg := new(global.LoggerConfig)
+//	for _, opt := range opts {
+//		opt(logCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.Logger = logCfg
+//	}
+//}
+
+func WithShutdown(opts ...graceful_shutdown.Option) InstanceOption {
+	sdOpts := graceful_shutdown.DefaultOptions()
+	for _, opt := range opts {
+		opt(sdOpts)
+	}
+
+	return func(insOpts *InstanceOptions) {
+		insOpts.Shutdown = sdOpts.Shutdown
+	}
+}
+
+// todo(DMwangnima): enumerate specific EventDispatcherType
+//func WithEventDispatcherType(typ string) InstanceOption {
+//	return func(cfg *InstanceOptions) {
+//		cfg.EventDispatcherType = typ
+//	}
+//}
+
+//func WithCacheFile(file string) InstanceOption {
+//	return func(cfg *InstanceOptions) {
+//		cfg.CacheFile = file
+//	}
+//}
+
+//func WithCustom(opts ...global.CustomOption) InstanceOption {
+//	cusCfg := new(global.CustomConfig)
+//	for _, opt := range opts {
+//		opt(cusCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.Custom = cusCfg
+//	}
+//}
+
+//func WithProfiles(opts ...global.ProfilesOption) InstanceOption {
+//	proCfg := new(global.ProfilesConfig)
+//	for _, opt := range opts {
+//		opt(proCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.Profiles = proCfg
+//	}
+//}
+
+//func WithTLS(opts ...global.TLSOption) InstanceOption {
+//	tlsCfg := new(global.TLSConfig)
+//	for _, opt := range opts {
+//		opt(tlsCfg)
+//	}
+//
+//	return func(cfg *InstanceOptions) {
+//		cfg.TLSConfig = tlsCfg
+//	}
+//}
diff --git a/protocol/dubbo3/dubbo3_protocol.go b/protocol/dubbo3/dubbo3_protocol.go
index 8d824b9..c3b6e7f 100644
--- a/protocol/dubbo3/dubbo3_protocol.go
+++ b/protocol/dubbo3/dubbo3_protocol.go
@@ -40,7 +40,6 @@
 import (
 	"dubbo.apache.org/dubbo-go/v3/common"
 	"dubbo.apache.org/dubbo-go/v3/common/constant"
-	"dubbo.apache.org/dubbo-go/v3/common/extension"
 	"dubbo.apache.org/dubbo-go/v3/config"
 	"dubbo.apache.org/dubbo-go/v3/protocol"
 	"dubbo.apache.org/dubbo-go/v3/protocol/invocation"
@@ -49,7 +48,8 @@
 var protocolOnce sync.Once
 
 func init() {
-	extension.SetProtocol(tripleConstant.TRIPLE, GetProtocol)
+	// todo(DMwangnima): deprecated
+	//extension.SetProtocol(tripleConstant.TRIPLE, GetProtocol)
 	protocolOnce = sync.Once{}
 }
 
diff --git a/protocol/invocation.go b/protocol/invocation.go
index a97cf72..cb98df6 100644
--- a/protocol/invocation.go
+++ b/protocol/invocation.go
@@ -34,6 +34,8 @@
 	ParameterTypes() []reflect.Type
 	// ParameterValues gets invocation parameter values.
 	ParameterValues() []reflect.Value
+	// ParameterRawValues gets raw invocation parameter values.
+	ParameterRawValues() []interface{}
 	// Arguments gets arguments.
 	Arguments() []interface{}
 	// Reply gets response of request
diff --git a/protocol/invocation/rpcinvocation.go b/protocol/invocation/rpcinvocation.go
index 5c821e6..10e1fa5 100644
--- a/protocol/invocation/rpcinvocation.go
+++ b/protocol/invocation/rpcinvocation.go
@@ -44,11 +44,12 @@
 	parameterTypeNames []string
 	parameterTypes     []reflect.Type
 
-	parameterValues []reflect.Value
-	arguments       []interface{}
-	reply           interface{}
-	callBack        interface{}
-	attachments     map[string]interface{}
+	parameterValues    []reflect.Value
+	parameterRawValues []interface{}
+	arguments          []interface{}
+	reply              interface{}
+	callBack           interface{}
+	attachments        map[string]interface{}
 	// Refer to dubbo 2.7.6.  It is different from attachment. It is used in internal process.
 	attributes map[string]interface{}
 	invoker    protocol.Invoker
@@ -114,6 +115,10 @@
 	return r.parameterValues
 }
 
+func (r *RPCInvocation) ParameterRawValues() []interface{} {
+	return r.parameterRawValues
+}
+
 // Arguments gets RPC arguments.
 func (r *RPCInvocation) Arguments() []interface{} {
 	return r.arguments
@@ -295,6 +300,13 @@
 	}
 }
 
+// WithParameterRawValues creates option with @parameterRawValues
+func WithParameterRawValues(parameterRawValues []interface{}) option {
+	return func(invo *RPCInvocation) {
+		invo.parameterRawValues = parameterRawValues
+	}
+}
+
 // WithArguments creates option with @arguments function.
 func WithArguments(arguments []interface{}) option {
 	return func(invo *RPCInvocation) {
diff --git a/protocol/options.go b/protocol/options.go
new file mode 100644
index 0000000..6373e59
--- /dev/null
+++ b/protocol/options.go
@@ -0,0 +1,96 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package protocol
+
+import (
+	"strconv"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+type Options struct {
+	Protocol *global.ProtocolConfig
+}
+
+func DefaultOptions() *Options {
+	return &Options{Protocol: global.DefaultProtocolConfig()}
+}
+
+type Option func(*Options)
+
+func WithDubbo() Option {
+	return func(opts *Options) {
+		opts.Protocol.Name = "dubbo"
+	}
+}
+
+func WithGRPC() Option {
+	return func(opts *Options) {
+		opts.Protocol.Name = "grpc"
+	}
+}
+
+func WithJSONRPC() Option {
+	return func(opts *Options) {
+		opts.Protocol.Name = "jsonrpc"
+	}
+}
+
+func WithREST() Option {
+	return func(opts *Options) {
+		opts.Protocol.Name = "rest"
+	}
+}
+
+func WithTriple() Option {
+	return func(opts *Options) {
+		opts.Protocol.Name = "tri"
+	}
+}
+
+func WithIp(ip string) Option {
+	return func(opts *Options) {
+		opts.Protocol.Ip = ip
+	}
+}
+
+func WithPort(port int) Option {
+	return func(opts *Options) {
+		opts.Protocol.Port = strconv.Itoa(port)
+	}
+}
+
+func WithParams(params interface{}) Option {
+	return func(opts *Options) {
+		opts.Protocol.Params = params
+	}
+}
+
+func WithMaxServerSendMsgSize(size int) Option {
+	return func(opts *Options) {
+		opts.Protocol.MaxServerSendMsgSize = strconv.Itoa(size)
+	}
+}
+
+func WithMaxServerRecvMsgSize(size int) Option {
+	return func(opts *Options) {
+		opts.Protocol.MaxServerRecvMsgSize = strconv.Itoa(size)
+	}
+}
diff --git a/protocol/triple/client.go b/protocol/triple/client.go
new file mode 100644
index 0000000..9d1f3fa
--- /dev/null
+++ b/protocol/triple/client.go
@@ -0,0 +1,242 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"context"
+	"crypto/tls"
+	"fmt"
+	"net"
+	"net/http"
+	url_package "net/url"
+	"strings"
+)
+
+import (
+	"github.com/dustin/go-humanize"
+
+	"golang.org/x/net/http2"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+)
+
+const (
+	httpPrefix  string = "http://"
+	httpsPrefix string = "https://"
+)
+
+// clientManager wraps triple clients and is responsible for find concrete triple client to invoke
+// callUnary, callClientStream, callServerStream, callBidiStream.
+// A Reference has a clientManager.
+type clientManager struct {
+	// triple_protocol clients, key is method name
+	triClients map[string]*tri.Client
+}
+
+func (cm *clientManager) getClient(method string) (*tri.Client, error) {
+	triClient, ok := cm.triClients[method]
+	if !ok {
+		return nil, fmt.Errorf("missing triple client for method: %s", method)
+	}
+	return triClient, nil
+}
+
+func (cm *clientManager) callUnary(ctx context.Context, method string, req, resp interface{}) error {
+	triClient, err := cm.getClient(method)
+	if err != nil {
+		return err
+	}
+	triReq := tri.NewRequest(req)
+	triResp := tri.NewResponse(resp)
+	if err := triClient.CallUnary(ctx, triReq, triResp); err != nil {
+		return err
+	}
+	return nil
+}
+
+func (cm *clientManager) callClientStream(ctx context.Context, method string) (interface{}, error) {
+	triClient, err := cm.getClient(method)
+	if err != nil {
+		return nil, err
+	}
+	stream, err := triClient.CallClientStream(ctx)
+	if err != nil {
+		return nil, err
+	}
+	return stream, nil
+}
+
+func (cm *clientManager) callServerStream(ctx context.Context, method string, req interface{}) (interface{}, error) {
+	triClient, err := cm.getClient(method)
+	if err != nil {
+		return nil, err
+	}
+	triReq := tri.NewRequest(req)
+	stream, err := triClient.CallServerStream(ctx, triReq)
+	if err != nil {
+		return nil, err
+	}
+	return stream, nil
+}
+
+func (cm *clientManager) callBidiStream(ctx context.Context, method string) (interface{}, error) {
+	triClient, err := cm.getClient(method)
+	if err != nil {
+		return nil, err
+	}
+	stream, err := triClient.CallBidiStream(ctx)
+	if err != nil {
+		return nil, err
+	}
+	return stream, nil
+}
+
+func (cm *clientManager) close() error {
+	// There is no need to release resources right now.
+	// But we leave this function here for future use.
+	return nil
+}
+
+func newClientManager(url *common.URL) (*clientManager, error) {
+	// If global trace instance was set, it means trace function enabled.
+	// If not, will return NoopTracer.
+	// tracer := opentracing.GlobalTracer()
+	var triClientOpts []tri.ClientOption
+
+	// set max send and recv msg size
+	maxCallRecvMsgSize := constant.DefaultMaxCallRecvMsgSize
+	if recvMsgSize, err := humanize.ParseBytes(url.GetParam(constant.MaxCallRecvMsgSize, "")); err == nil && recvMsgSize > 0 {
+		maxCallRecvMsgSize = int(recvMsgSize)
+	}
+	triClientOpts = append(triClientOpts, tri.WithReadMaxBytes(maxCallRecvMsgSize))
+	maxCallSendMsgSize := constant.DefaultMaxCallSendMsgSize
+	if sendMsgSize, err := humanize.ParseBytes(url.GetParam(constant.MaxCallSendMsgSize, "")); err == nil && sendMsgSize > 0 {
+		maxCallSendMsgSize = int(sendMsgSize)
+	}
+	triClientOpts = append(triClientOpts, tri.WithSendMaxBytes(maxCallSendMsgSize))
+
+	// set serialization
+	serialization := url.GetParam(constant.SerializationKey, constant.ProtobufSerialization)
+	switch serialization {
+	case constant.ProtobufSerialization:
+	case constant.JSONSerialization:
+		triClientOpts = append(triClientOpts, tri.WithProtoJSON())
+	default:
+		panic(fmt.Sprintf("Unsupported serialization: %s", serialization))
+	}
+
+	// todo:// process timeout
+	// consumer config client connectTimeout
+	//connectTimeout := config.GetConsumerConfig().ConnectTimeout
+
+	// dialOpts = append(dialOpts,
+	//
+	//	grpc.WithBlock(),
+	//	// todo config network timeout
+	//	grpc.WithTimeout(time.Second*3),
+	//	grpc.WithUnaryInterceptor(otgrpc.OpenTracingClientInterceptor(tracer, otgrpc.LogPayloads())),
+	//	grpc.WithStreamInterceptor(otgrpc.OpenTracingStreamClientInterceptor(tracer, otgrpc.LogPayloads())),
+	//	grpc.WithDefaultCallOptions(
+	//		grpc.CallContentSubtype(clientConf.ContentSubType),
+	//		grpc.MaxCallRecvMsgSize(maxCallRecvMsgSize),
+	//		grpc.MaxCallSendMsgSize(maxCallSendMsgSize),
+	//	),
+	//
+	// )
+	var cfg *tls.Config
+	var tlsFlag bool
+	//var err error
+
+	// todo: think about a more elegant way to configure tls
+	//if tlsConfig := config.GetRootConfig().TLSConfig; tlsConfig != nil {
+	//	cfg, err = config.GetClientTlsConfig(&config.TLSConfig{
+	//		CACertFile:    tlsConfig.CACertFile,
+	//		TLSCertFile:   tlsConfig.TLSCertFile,
+	//		TLSKeyFile:    tlsConfig.TLSKeyFile,
+	//		TLSServerName: tlsConfig.TLSServerName,
+	//	})
+	//	if err != nil {
+	//		return nil, err
+	//	}
+	//	logger.Infof("TRIPLE clientManager initialized the TLSConfig configuration successfully")
+	//	tlsFlag = true
+	//}
+
+	// todo(DMwangnima): this code fragment would be used to be compatible with old triple client
+	//key := url.GetParam(constant.InterfaceKey, "")
+	//conRefs := config.GetConsumerConfig().References
+	//ref, ok := conRefs[key]
+	//if !ok {
+	//	panic("no reference")
+	//}
+	// todo: set timeout
+	var transport http.RoundTripper
+	callType := url.GetParam(constant.CallHTTPTypeKey, constant.CallHTTP2)
+	switch callType {
+	case constant.CallHTTP:
+		transport = &http.Transport{
+			TLSClientConfig: cfg,
+		}
+		triClientOpts = append(triClientOpts, tri.WithTriple())
+	case constant.CallHTTP2:
+		if tlsFlag {
+			transport = &http2.Transport{
+				TLSClientConfig: cfg,
+			}
+		} else {
+			transport = &http2.Transport{
+				DialTLSContext: func(_ context.Context, network, addr string, _ *tls.Config) (net.Conn, error) {
+					return net.Dial(network, addr)
+				},
+				AllowHTTP: true,
+			}
+		}
+		triClientOpts = append(triClientOpts)
+	default:
+		panic(fmt.Sprintf("Unsupported callType: %s", callType))
+	}
+	httpClient := &http.Client{
+		Transport: transport,
+	}
+
+	var baseTriURL string
+	baseTriURL = strings.TrimPrefix(url.Location, httpPrefix)
+	baseTriURL = strings.TrimPrefix(url.Location, httpsPrefix)
+	if tlsFlag {
+		baseTriURL = httpsPrefix + baseTriURL
+	} else {
+		baseTriURL = httpPrefix + baseTriURL
+	}
+	triClients := make(map[string]*tri.Client)
+	for _, method := range url.Methods {
+		triURL, err := url_package.JoinPath(baseTriURL, url.Interface(), method)
+		if err != nil {
+			return nil, fmt.Errorf("JoinPath failed for base %s, interface %s, method %s", baseTriURL, url.Interface(), method)
+		}
+		triClient := tri.NewClient(httpClient, triURL, triClientOpts...)
+		triClients[method] = triClient
+	}
+
+	return &clientManager{
+		triClients: triClients,
+	}, nil
+}
diff --git a/protocol/triple/codec.go b/protocol/triple/codec.go
new file mode 100644
index 0000000..a0e1a75
--- /dev/null
+++ b/protocol/triple/codec.go
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"bytes"
+	"encoding/json"
+)
+
+import (
+	"github.com/golang/protobuf/jsonpb"
+	"github.com/golang/protobuf/proto"
+
+	"google.golang.org/grpc/encoding"
+)
+
+const (
+	codecJson  = "json"
+	codecProto = "proto"
+)
+
+func init() {
+	encoding.RegisterCodec(grpcJson{
+		Marshaler: jsonpb.Marshaler{
+			EmitDefaults: true,
+			OrigName:     true,
+		},
+	})
+}
+
+type grpcJson struct {
+	jsonpb.Marshaler
+	jsonpb.Unmarshaler
+}
+
+// Name implements grpc encoding package Codec interface method,
+// returns the name of the Codec implementation.
+func (_ grpcJson) Name() string {
+	return codecJson
+}
+
+// Marshal implements grpc encoding package Codec interface method,returns the wire format of v.
+func (j grpcJson) Marshal(v interface{}) (out []byte, err error) {
+	if pm, ok := v.(proto.Message); ok {
+		b := new(bytes.Buffer)
+		err := j.Marshaler.Marshal(b, pm)
+		if err != nil {
+			return nil, err
+		}
+		return b.Bytes(), nil
+	}
+	return json.Marshal(v)
+}
+
+// Unmarshal implements grpc encoding package Codec interface method,Unmarshal parses the wire format into v.
+func (j grpcJson) Unmarshal(data []byte, v interface{}) (err error) {
+	if pm, ok := v.(proto.Message); ok {
+		b := bytes.NewBuffer(data)
+		return j.Unmarshaler.Unmarshal(b, pm)
+	}
+	return json.Unmarshal(data, v)
+}
diff --git a/protocol/triple/common.go b/protocol/triple/common.go
new file mode 100644
index 0000000..d84a853
--- /dev/null
+++ b/protocol/triple/common.go
@@ -0,0 +1,27 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"net/url"
+)
+
+func joinProcedure(interfaceName, methodName string) string {
+	procedure, _ := url.JoinPath("", interfaceName, methodName)
+	return "/" + procedure
+}
diff --git a/protocol/triple/internal/client/cmd_client/main.go b/protocol/triple/internal/client/cmd_client/main.go
new file mode 100644
index 0000000..7adb0fa
--- /dev/null
+++ b/protocol/triple/internal/client/cmd_client/main.go
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/client"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/client/common"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+)
+
+func main() {
+	// for the most brief RPC case
+	cli, err := client.NewClient(
+		client.WithURL("tri://127.0.0.1:20000"),
+	)
+	if err != nil {
+		panic(err)
+	}
+	svc, err := greettriple.NewGreetService(cli)
+	if err != nil {
+		panic(err)
+	}
+
+	common.TestClient(svc)
+}
diff --git a/protocol/triple/internal/client/cmd_client_with_registry/main.go b/protocol/triple/internal/client/cmd_client_with_registry/main.go
new file mode 100644
index 0000000..4d2acb9
--- /dev/null
+++ b/protocol/triple/internal/client/cmd_client_with_registry/main.go
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/client"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/client/common"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+func main() {
+	// for the most brief RPC case with Registry
+
+	cli, err := client.NewClient(
+		client.WithRegistry("zk",
+			registry.WithZookeeper(),
+			registry.WithAddress("127.0.0.1:2181"),
+		),
+	)
+	if err != nil {
+		panic(err)
+	}
+	svc, err := greettriple.NewGreetService(cli)
+	if err != nil {
+		panic(err)
+	}
+
+	common.TestClient(svc)
+}
diff --git a/protocol/triple/internal/client/cmd_instance/main.go b/protocol/triple/internal/client/cmd_instance/main.go
new file mode 100644
index 0000000..262ca49
--- /dev/null
+++ b/protocol/triple/internal/client/cmd_instance/main.go
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3"
+	"dubbo.apache.org/dubbo-go/v3/client"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/client/common"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+)
+
+func main() {
+	// global conception
+	// configure global configurations and common modules
+	ins, err := dubbo.NewInstance(
+		dubbo.WithName("dubbo_test"),
+	)
+	if err != nil {
+		panic(err)
+	}
+	// configure the params that only client layer cares
+	cli, err := ins.NewClient(
+		client.WithURL("tri://127.0.0.1:20000"),
+	)
+	if err != nil {
+		panic(err)
+	}
+
+	svc, err := greettriple.NewGreetService(cli)
+	if err != nil {
+		panic(err)
+	}
+
+	common.TestClient(svc)
+}
diff --git a/protocol/triple/internal/client/cmd_instance_with_registry/main.go b/protocol/triple/internal/client/cmd_instance_with_registry/main.go
new file mode 100644
index 0000000..38f9d3a
--- /dev/null
+++ b/protocol/triple/internal/client/cmd_instance_with_registry/main.go
@@ -0,0 +1,56 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3"
+	"dubbo.apache.org/dubbo-go/v3/client"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/client/common"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+func main() {
+	// global conception
+	// configure global configurations and common modules
+	ins, err := dubbo.NewInstance(
+		dubbo.WithName("dubbo_test"),
+		dubbo.WithRegistry("zk",
+			registry.WithZookeeper(),
+			registry.WithAddress("127.0.0.1:2181"),
+		),
+	)
+	if err != nil {
+		panic(err)
+	}
+	// configure the params that only client layer cares
+	cli, err := ins.NewClient(
+		client.WithRegistryIDs([]string{"zk"}),
+	)
+	if err != nil {
+		panic(err)
+	}
+
+	svc, err := greettriple.NewGreetService(cli)
+	if err != nil {
+		panic(err)
+	}
+
+	common.TestClient(svc)
+}
diff --git a/protocol/triple/internal/client/common/client.go b/protocol/triple/internal/client/common/client.go
new file mode 100644
index 0000000..af7f257
--- /dev/null
+++ b/protocol/triple/internal/client/common/client.go
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package common
+
+import (
+	"context"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	greet "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+)
+
+func TestClient(cli greettriple.GreetService) {
+	if err := testUnary(cli); err != nil {
+		logger.Error(err)
+	}
+
+	if err := testBidiStream(cli); err != nil {
+		logger.Error(err)
+	}
+
+	if err := testClientStream(cli); err != nil {
+		logger.Error(err)
+	}
+
+	if err := testServerStream(cli); err != nil {
+		logger.Error(err)
+	}
+}
+
+func testUnary(cli greettriple.GreetService) error {
+	logger.Info("start to test TRIPLE unary call")
+	resp, err := cli.Greet(context.Background(), &greet.GreetRequest{Name: "triple"})
+	if err != nil {
+		return err
+	}
+	logger.Infof("TRIPLE unary call resp: %s", resp.Greeting)
+	return nil
+}
+
+func testBidiStream(cli greettriple.GreetService) error {
+	logger.Info("start to test TRIPLE bidi stream")
+	stream, err := cli.GreetStream(context.Background())
+	if err != nil {
+		return err
+	}
+	if err := stream.Send(&greet.GreetStreamRequest{Name: "triple"}); err != nil {
+		return err
+	}
+	resp, err := stream.Recv()
+	if err != nil {
+		return err
+	}
+	logger.Infof("TRIPLE bidi stream resp: %s", resp.Greeting)
+	if err := stream.CloseRequest(); err != nil {
+		return err
+	}
+	if err := stream.CloseResponse(); err != nil {
+		return err
+	}
+	return nil
+}
+
+func testClientStream(cli greettriple.GreetService) error {
+	logger.Info("start to test TRIPLE client stream")
+	stream, err := cli.GreetClientStream(context.Background())
+	if err != nil {
+		return err
+	}
+	for i := 0; i < 5; i++ {
+		if err := stream.Send(&greet.GreetClientStreamRequest{Name: "triple"}); err != nil {
+			return err
+		}
+	}
+	resp, err := stream.CloseAndRecv()
+	if err != nil {
+		return err
+	}
+	logger.Infof("TRIPLE client stream resp: %s", resp.Greeting)
+	return nil
+}
+
+func testServerStream(cli greettriple.GreetService) error {
+	logger.Info("start to test TRIPLE server stream")
+	stream, err := cli.GreetServerStream(context.Background(), &greet.GreetServerStreamRequest{Name: "triple"})
+	if err != nil {
+		return err
+	}
+	for stream.Recv() {
+		logger.Infof("TRIPLE server stream resp: %s", stream.Msg().Greeting)
+	}
+	if stream.Err() != nil {
+		return err
+	}
+	if err := stream.Close(); err != nil {
+		return err
+	}
+	return nil
+}
diff --git a/protocol/triple/internal/dubbo3_server/api/greet_service.go b/protocol/triple/internal/dubbo3_server/api/greet_service.go
new file mode 100644
index 0000000..81043f9
--- /dev/null
+++ b/protocol/triple/internal/dubbo3_server/api/greet_service.go
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package api
+
+import (
+	"context"
+	"fmt"
+	"io"
+	"strings"
+)
+
+import (
+	"github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+	greet "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/dubbo3_gen"
+)
+
+type GreetDubbo3Server struct {
+	greet.UnimplementedGreetServiceServer
+}
+
+func (srv *GreetDubbo3Server) Greet(ctx context.Context, req *proto.GreetRequest) (*proto.GreetResponse, error) {
+	return &proto.GreetResponse{Greeting: req.Name}, nil
+}
+
+func (srv *GreetDubbo3Server) GreetStream(stream greet.GreetService_GreetStreamServer) error {
+	for {
+		req, err := stream.Recv()
+		if err != nil {
+			if errors.Is(err, io.EOF) {
+				break
+			}
+			return fmt.Errorf("dubbo3 Bidistream recv error: %s", err)
+		}
+		if err := stream.Send(&proto.GreetStreamResponse{Greeting: req.Name}); err != nil {
+			return fmt.Errorf("dubbo3 Bidistream send error: %s", err)
+		}
+	}
+	return nil
+}
+
+func (srv *GreetDubbo3Server) GreetClientStream(stream greet.GreetService_GreetClientStreamServer) error {
+	var reqs []string
+	for {
+		req, err := stream.Recv()
+		if err != nil {
+			if errors.Is(err, io.EOF) {
+				break
+			}
+			return fmt.Errorf("dubbo3 ClientStream recv error: %s", err)
+		}
+		reqs = append(reqs, req.Name)
+	}
+
+	resp := &proto.GreetClientStreamResponse{
+		Greeting: strings.Join(reqs, ","),
+	}
+	return stream.SendAndClose(resp)
+}
+
+func (srv *GreetDubbo3Server) GreetServerStream(req *proto.GreetServerStreamRequest, stream greet.GreetService_GreetServerStreamServer) error {
+	for i := 0; i < 5; i++ {
+		if err := stream.Send(&proto.GreetServerStreamResponse{Greeting: req.Name}); err != nil {
+			return fmt.Errorf("dubbo3 ServerStream send error: %s", err)
+		}
+	}
+	return nil
+}
diff --git a/protocol/triple/internal/dubbo3_server/cmd/dubbogo.yml b/protocol/triple/internal/dubbo3_server/cmd/dubbogo.yml
new file mode 100644
index 0000000..a3a562f
--- /dev/null
+++ b/protocol/triple/internal/dubbo3_server/cmd/dubbogo.yml
@@ -0,0 +1,13 @@
+dubbo:
+#  registries:
+#    zk:
+#      address: zookeeper://127.0.0.1:2181
+  protocols:
+    triple:
+      name: tri
+      port: 20001
+  provider:
+    services:
+      GreetDubbo3Server:
+        # interface is for registry
+        interface: greet.GreetService
\ No newline at end of file
diff --git a/protocol/triple/internal/dubbo3_server/cmd/main.go b/protocol/triple/internal/dubbo3_server/cmd/main.go
new file mode 100644
index 0000000..aac22da
--- /dev/null
+++ b/protocol/triple/internal/dubbo3_server/cmd/main.go
@@ -0,0 +1,15 @@
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/config"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/dubbo3_server/api"
+)
+
+func main() {
+	config.SetProviderService(&api.GreetDubbo3Server{})
+	if err := config.Load(config.WithPath("./dubbogo.yml")); err != nil {
+		panic(err)
+	}
+	select {}
+}
diff --git a/protocol/triple/internal/proto/dubbo3_gen/greet_triple.pb.go b/protocol/triple/internal/proto/dubbo3_gen/greet_triple.pb.go
new file mode 100644
index 0000000..030bb7a
--- /dev/null
+++ b/protocol/triple/internal/proto/dubbo3_gen/greet_triple.pb.go
@@ -0,0 +1,382 @@
+// Code generated by protoc-gen-go-triple. DO NOT EDIT.
+// versions:
+// - protoc-gen-go-triple v1.0.5
+// - protoc             v3.20.3
+// source: greet.proto
+
+package greet
+
+import (
+	context "context"
+	fmt "fmt"
+)
+
+import (
+	grpc_go "github.com/dubbogo/grpc-go"
+	codes "github.com/dubbogo/grpc-go/codes"
+	metadata "github.com/dubbogo/grpc-go/metadata"
+	status "github.com/dubbogo/grpc-go/status"
+
+	common "github.com/dubbogo/triple/pkg/common"
+	constant "github.com/dubbogo/triple/pkg/common/constant"
+	triple "github.com/dubbogo/triple/pkg/triple"
+)
+
+import (
+	protocol "dubbo.apache.org/dubbo-go/v3/protocol"
+	dubbo3 "dubbo.apache.org/dubbo-go/v3/protocol/dubbo3"
+	invocation "dubbo.apache.org/dubbo-go/v3/protocol/invocation"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+)
+
+// This is a compile-time assertion to ensure that this generated file
+// is compatible with the grpc package it is being compiled against.
+const _ = grpc_go.SupportPackageIsVersion7
+
+// GreetServiceClient is the client API for GreetService service.
+//
+// For semantics around ctx use and closing/ending streaming RPCs, please refer to https://pkg.go.dev/google.golang.org/grpc/?tab=doc#ClientConn.NewStream.
+type GreetServiceClient interface {
+	Greet(ctx context.Context, in *proto.GreetRequest, opts ...grpc_go.CallOption) (*proto.GreetResponse, common.ErrorWithAttachment)
+	GreetStream(ctx context.Context, opts ...grpc_go.CallOption) (GreetService_GreetStreamClient, error)
+	GreetClientStream(ctx context.Context, opts ...grpc_go.CallOption) (GreetService_GreetClientStreamClient, error)
+	GreetServerStream(ctx context.Context, in *proto.GreetServerStreamRequest, opts ...grpc_go.CallOption) (GreetService_GreetServerStreamClient, error)
+}
+
+type greetServiceClient struct {
+	cc *triple.TripleConn
+}
+
+type GreetServiceClientImpl struct {
+	Greet             func(ctx context.Context, in *proto.GreetRequest) (*proto.GreetResponse, error)
+	GreetStream       func(ctx context.Context) (GreetService_GreetStreamClient, error)
+	GreetClientStream func(ctx context.Context) (GreetService_GreetClientStreamClient, error)
+	GreetServerStream func(ctx context.Context, in *proto.GreetServerStreamRequest) (GreetService_GreetServerStreamClient, error)
+}
+
+func (c *GreetServiceClientImpl) GetDubboStub(cc *triple.TripleConn) GreetServiceClient {
+	return NewGreetServiceClient(cc)
+}
+
+func (c *GreetServiceClientImpl) XXX_InterfaceName() string {
+	return "greet.GreetService"
+}
+
+func NewGreetServiceClient(cc *triple.TripleConn) GreetServiceClient {
+	return &greetServiceClient{cc}
+}
+
+func (c *greetServiceClient) Greet(ctx context.Context, in *proto.GreetRequest, opts ...grpc_go.CallOption) (*proto.GreetResponse, common.ErrorWithAttachment) {
+	out := new(proto.GreetResponse)
+	interfaceKey := ctx.Value(constant.InterfaceKey).(string)
+	return out, c.cc.Invoke(ctx, "/"+interfaceKey+"/Greet", in, out)
+}
+
+func (c *greetServiceClient) GreetStream(ctx context.Context, opts ...grpc_go.CallOption) (GreetService_GreetStreamClient, error) {
+	interfaceKey := ctx.Value(constant.InterfaceKey).(string)
+	stream, err := c.cc.NewStream(ctx, "/"+interfaceKey+"/GreetStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &greetServiceGreetStreamClient{stream}
+	return x, nil
+}
+
+type GreetService_GreetStreamClient interface {
+	Send(*proto.GreetStreamRequest) error
+	Recv() (*proto.GreetStreamResponse, error)
+	grpc_go.ClientStream
+}
+
+type greetServiceGreetStreamClient struct {
+	grpc_go.ClientStream
+}
+
+func (x *greetServiceGreetStreamClient) Send(m *proto.GreetStreamRequest) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *greetServiceGreetStreamClient) Recv() (*proto.GreetStreamResponse, error) {
+	m := new(proto.GreetStreamResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *greetServiceClient) GreetClientStream(ctx context.Context, opts ...grpc_go.CallOption) (GreetService_GreetClientStreamClient, error) {
+	interfaceKey := ctx.Value(constant.InterfaceKey).(string)
+	stream, err := c.cc.NewStream(ctx, "/"+interfaceKey+"/GreetClientStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &greetServiceGreetClientStreamClient{stream}
+	return x, nil
+}
+
+type GreetService_GreetClientStreamClient interface {
+	Send(*proto.GreetClientStreamRequest) error
+	CloseAndRecv() (*proto.GreetClientStreamResponse, error)
+	grpc_go.ClientStream
+}
+
+type greetServiceGreetClientStreamClient struct {
+	grpc_go.ClientStream
+}
+
+func (x *greetServiceGreetClientStreamClient) Send(m *proto.GreetClientStreamRequest) error {
+	return x.ClientStream.SendMsg(m)
+}
+
+func (x *greetServiceGreetClientStreamClient) CloseAndRecv() (*proto.GreetClientStreamResponse, error) {
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	m := new(proto.GreetClientStreamResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func (c *greetServiceClient) GreetServerStream(ctx context.Context, in *proto.GreetServerStreamRequest, opts ...grpc_go.CallOption) (GreetService_GreetServerStreamClient, error) {
+	interfaceKey := ctx.Value(constant.InterfaceKey).(string)
+	stream, err := c.cc.NewStream(ctx, "/"+interfaceKey+"/GreetServerStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	x := &greetServiceGreetServerStreamClient{stream}
+	if err := x.ClientStream.SendMsg(in); err != nil {
+		return nil, err
+	}
+	if err := x.ClientStream.CloseSend(); err != nil {
+		return nil, err
+	}
+	return x, nil
+}
+
+type GreetService_GreetServerStreamClient interface {
+	Recv() (*proto.GreetServerStreamResponse, error)
+	grpc_go.ClientStream
+}
+
+type greetServiceGreetServerStreamClient struct {
+	grpc_go.ClientStream
+}
+
+func (x *greetServiceGreetServerStreamClient) Recv() (*proto.GreetServerStreamResponse, error) {
+	m := new(proto.GreetServerStreamResponse)
+	if err := x.ClientStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+// GreetServiceServer is the server API for GreetService service.
+// All implementations must embed UnimplementedGreetServiceServer
+// for forward compatibility
+type GreetServiceServer interface {
+	Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error)
+	GreetStream(GreetService_GreetStreamServer) error
+	GreetClientStream(GreetService_GreetClientStreamServer) error
+	GreetServerStream(*proto.GreetServerStreamRequest, GreetService_GreetServerStreamServer) error
+	mustEmbedUnimplementedGreetServiceServer()
+}
+
+// UnimplementedGreetServiceServer must be embedded to have forward compatible implementations.
+type UnimplementedGreetServiceServer struct {
+	proxyImpl protocol.Invoker
+}
+
+func (UnimplementedGreetServiceServer) Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error) {
+	return nil, status.Errorf(codes.Unimplemented, "method Greet not implemented")
+}
+func (UnimplementedGreetServiceServer) GreetStream(GreetService_GreetStreamServer) error {
+	return status.Errorf(codes.Unimplemented, "method GreetStream not implemented")
+}
+func (UnimplementedGreetServiceServer) GreetClientStream(GreetService_GreetClientStreamServer) error {
+	return status.Errorf(codes.Unimplemented, "method GreetClientStream not implemented")
+}
+func (UnimplementedGreetServiceServer) GreetServerStream(*proto.GreetServerStreamRequest, GreetService_GreetServerStreamServer) error {
+	return status.Errorf(codes.Unimplemented, "method GreetServerStream not implemented")
+}
+func (s *UnimplementedGreetServiceServer) XXX_SetProxyImpl(impl protocol.Invoker) {
+	s.proxyImpl = impl
+}
+
+func (s *UnimplementedGreetServiceServer) XXX_GetProxyImpl() protocol.Invoker {
+	return s.proxyImpl
+}
+
+func (s *UnimplementedGreetServiceServer) XXX_ServiceDesc() *grpc_go.ServiceDesc {
+	return &GreetService_ServiceDesc
+}
+func (s *UnimplementedGreetServiceServer) XXX_InterfaceName() string {
+	return "greet.GreetService"
+}
+
+func (UnimplementedGreetServiceServer) mustEmbedUnimplementedGreetServiceServer() {}
+
+// UnsafeGreetServiceServer may be embedded to opt out of forward compatibility for this service.
+// Use of this interface is not recommended, as added methods to GreetServiceServer will
+// result in compilation errors.
+type UnsafeGreetServiceServer interface {
+	mustEmbedUnimplementedGreetServiceServer()
+}
+
+func RegisterGreetServiceServer(s grpc_go.ServiceRegistrar, srv GreetServiceServer) {
+	s.RegisterService(&GreetService_ServiceDesc, srv)
+}
+
+func _GreetService_Greet_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc_go.UnaryServerInterceptor) (interface{}, error) {
+	in := new(proto.GreetRequest)
+	if err := dec(in); err != nil {
+		return nil, err
+	}
+	base := srv.(dubbo3.Dubbo3GrpcService)
+	args := []interface{}{}
+	args = append(args, in)
+	md, _ := metadata.FromIncomingContext(ctx)
+	invAttachment := make(map[string]interface{}, len(md))
+	for k, v := range md {
+		invAttachment[k] = v
+	}
+	invo := invocation.NewRPCInvocation("Greet", args, invAttachment)
+	if interceptor == nil {
+		result := base.XXX_GetProxyImpl().Invoke(ctx, invo)
+		return result, result.Error()
+	}
+	info := &grpc_go.UnaryServerInfo{
+		Server:     srv,
+		FullMethod: ctx.Value("XXX_TRIPLE_GO_INTERFACE_NAME").(string),
+	}
+	handler := func(ctx context.Context, req interface{}) (interface{}, error) {
+		result := base.XXX_GetProxyImpl().Invoke(ctx, invo)
+		return result, result.Error()
+	}
+	return interceptor(ctx, in, info, handler)
+}
+
+func _GreetService_GreetStream_Handler(srv interface{}, stream grpc_go.ServerStream) error {
+	_, ok := srv.(dubbo3.Dubbo3GrpcService)
+	invo := invocation.NewRPCInvocation("GreetStream", nil, nil)
+	if !ok {
+		fmt.Println(invo)
+		return nil
+	}
+	return srv.(GreetServiceServer).GreetStream(&greetServiceGreetStreamServer{stream})
+}
+
+type GreetService_GreetStreamServer interface {
+	Send(*proto.GreetStreamResponse) error
+	Recv() (*proto.GreetStreamRequest, error)
+	grpc_go.ServerStream
+}
+
+type greetServiceGreetStreamServer struct {
+	grpc_go.ServerStream
+}
+
+func (x *greetServiceGreetStreamServer) Send(m *proto.GreetStreamResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *greetServiceGreetStreamServer) Recv() (*proto.GreetStreamRequest, error) {
+	m := new(proto.GreetStreamRequest)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func _GreetService_GreetClientStream_Handler(srv interface{}, stream grpc_go.ServerStream) error {
+	_, ok := srv.(dubbo3.Dubbo3GrpcService)
+	invo := invocation.NewRPCInvocation("GreetClientStream", nil, nil)
+	if !ok {
+		fmt.Println(invo)
+		return nil
+	}
+	return srv.(GreetServiceServer).GreetClientStream(&greetServiceGreetClientStreamServer{stream})
+}
+
+type GreetService_GreetClientStreamServer interface {
+	SendAndClose(*proto.GreetClientStreamResponse) error
+	Recv() (*proto.GreetClientStreamRequest, error)
+	grpc_go.ServerStream
+}
+
+type greetServiceGreetClientStreamServer struct {
+	grpc_go.ServerStream
+}
+
+func (x *greetServiceGreetClientStreamServer) SendAndClose(m *proto.GreetClientStreamResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+func (x *greetServiceGreetClientStreamServer) Recv() (*proto.GreetClientStreamRequest, error) {
+	m := new(proto.GreetClientStreamRequest)
+	if err := x.ServerStream.RecvMsg(m); err != nil {
+		return nil, err
+	}
+	return m, nil
+}
+
+func _GreetService_GreetServerStream_Handler(srv interface{}, stream grpc_go.ServerStream) error {
+	_, ok := srv.(dubbo3.Dubbo3GrpcService)
+	invo := invocation.NewRPCInvocation("GreetServerStream", nil, nil)
+	if !ok {
+		fmt.Println(invo)
+		return nil
+	}
+	m := new(proto.GreetServerStreamRequest)
+	if err := stream.RecvMsg(m); err != nil {
+		return err
+	}
+	return srv.(GreetServiceServer).GreetServerStream(m, &greetServiceGreetServerStreamServer{stream})
+}
+
+type GreetService_GreetServerStreamServer interface {
+	Send(*proto.GreetServerStreamResponse) error
+	grpc_go.ServerStream
+}
+
+type greetServiceGreetServerStreamServer struct {
+	grpc_go.ServerStream
+}
+
+func (x *greetServiceGreetServerStreamServer) Send(m *proto.GreetServerStreamResponse) error {
+	return x.ServerStream.SendMsg(m)
+}
+
+// GreetService_ServiceDesc is the grpc_go.ServiceDesc for GreetService service.
+// It's only intended for direct use with grpc_go.RegisterService,
+// and not to be introspected or modified (even as a copy)
+var GreetService_ServiceDesc = grpc_go.ServiceDesc{
+	ServiceName: "greet.GreetService",
+	HandlerType: (*GreetServiceServer)(nil),
+	Methods: []grpc_go.MethodDesc{
+		{
+			MethodName: "Greet",
+			Handler:    _GreetService_Greet_Handler,
+		},
+	},
+	Streams: []grpc_go.StreamDesc{
+		{
+			StreamName:    "GreetStream",
+			Handler:       _GreetService_GreetStream_Handler,
+			ServerStreams: true,
+			ClientStreams: true,
+		},
+		{
+			StreamName:    "GreetClientStream",
+			Handler:       _GreetService_GreetClientStream_Handler,
+			ClientStreams: true,
+		},
+		{
+			StreamName:    "GreetServerStream",
+			Handler:       _GreetService_GreetServerStream_Handler,
+			ServerStreams: true,
+		},
+	},
+	Metadata: "greet.proto",
+}
diff --git a/protocol/triple/internal/proto/greet.pb.go b/protocol/triple/internal/proto/greet.pb.go
new file mode 100644
index 0000000..15f94f5
--- /dev/null
+++ b/protocol/triple/internal/proto/greet.pb.go
@@ -0,0 +1,620 @@
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        v3.20.3
+// source: greet.proto
+
+package proto
+
+import (
+	reflect "reflect"
+	sync "sync"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type GreetRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (x *GreetRequest) Reset() {
+	*x = GreetRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetRequest) ProtoMessage() {}
+
+func (x *GreetRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetRequest.ProtoReflect.Descriptor instead.
+func (*GreetRequest) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *GreetRequest) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+type GreetResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Greeting string `protobuf:"bytes,1,opt,name=greeting,proto3" json:"greeting,omitempty"`
+}
+
+func (x *GreetResponse) Reset() {
+	*x = GreetResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetResponse) ProtoMessage() {}
+
+func (x *GreetResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetResponse.ProtoReflect.Descriptor instead.
+func (*GreetResponse) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *GreetResponse) GetGreeting() string {
+	if x != nil {
+		return x.Greeting
+	}
+	return ""
+}
+
+type GreetStreamRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (x *GreetStreamRequest) Reset() {
+	*x = GreetStreamRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetStreamRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetStreamRequest) ProtoMessage() {}
+
+func (x *GreetStreamRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetStreamRequest.ProtoReflect.Descriptor instead.
+func (*GreetStreamRequest) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *GreetStreamRequest) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+type GreetStreamResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Greeting string `protobuf:"bytes,1,opt,name=greeting,proto3" json:"greeting,omitempty"`
+}
+
+func (x *GreetStreamResponse) Reset() {
+	*x = GreetStreamResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetStreamResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetStreamResponse) ProtoMessage() {}
+
+func (x *GreetStreamResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetStreamResponse.ProtoReflect.Descriptor instead.
+func (*GreetStreamResponse) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{3}
+}
+
+func (x *GreetStreamResponse) GetGreeting() string {
+	if x != nil {
+		return x.Greeting
+	}
+	return ""
+}
+
+type GreetClientStreamRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (x *GreetClientStreamRequest) Reset() {
+	*x = GreetClientStreamRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetClientStreamRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetClientStreamRequest) ProtoMessage() {}
+
+func (x *GreetClientStreamRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetClientStreamRequest.ProtoReflect.Descriptor instead.
+func (*GreetClientStreamRequest) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{4}
+}
+
+func (x *GreetClientStreamRequest) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+type GreetClientStreamResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Greeting string `protobuf:"bytes,1,opt,name=greeting,proto3" json:"greeting,omitempty"`
+}
+
+func (x *GreetClientStreamResponse) Reset() {
+	*x = GreetClientStreamResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetClientStreamResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetClientStreamResponse) ProtoMessage() {}
+
+func (x *GreetClientStreamResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetClientStreamResponse.ProtoReflect.Descriptor instead.
+func (*GreetClientStreamResponse) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{5}
+}
+
+func (x *GreetClientStreamResponse) GetGreeting() string {
+	if x != nil {
+		return x.Greeting
+	}
+	return ""
+}
+
+type GreetServerStreamRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Name string `protobuf:"bytes,1,opt,name=name,proto3" json:"name,omitempty"`
+}
+
+func (x *GreetServerStreamRequest) Reset() {
+	*x = GreetServerStreamRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetServerStreamRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetServerStreamRequest) ProtoMessage() {}
+
+func (x *GreetServerStreamRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetServerStreamRequest.ProtoReflect.Descriptor instead.
+func (*GreetServerStreamRequest) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{6}
+}
+
+func (x *GreetServerStreamRequest) GetName() string {
+	if x != nil {
+		return x.Name
+	}
+	return ""
+}
+
+type GreetServerStreamResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Greeting string `protobuf:"bytes,1,opt,name=greeting,proto3" json:"greeting,omitempty"`
+}
+
+func (x *GreetServerStreamResponse) Reset() {
+	*x = GreetServerStreamResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_greet_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *GreetServerStreamResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*GreetServerStreamResponse) ProtoMessage() {}
+
+func (x *GreetServerStreamResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_greet_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use GreetServerStreamResponse.ProtoReflect.Descriptor instead.
+func (*GreetServerStreamResponse) Descriptor() ([]byte, []int) {
+	return file_greet_proto_rawDescGZIP(), []int{7}
+}
+
+func (x *GreetServerStreamResponse) GetGreeting() string {
+	if x != nil {
+		return x.Greeting
+	}
+	return ""
+}
+
+var File_greet_proto protoreflect.FileDescriptor
+
+var file_greet_proto_rawDesc = []byte{
+	0x0a, 0x0b, 0x67, 0x72, 0x65, 0x65, 0x74, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x05, 0x67,
+	0x72, 0x65, 0x65, 0x74, 0x22, 0x22, 0x0a, 0x0c, 0x47, 0x72, 0x65, 0x65, 0x74, 0x52, 0x65, 0x71,
+	0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x2b, 0x0a, 0x0d, 0x47, 0x72, 0x65, 0x65,
+	0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x67, 0x72, 0x65,
+	0x65, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x67, 0x72, 0x65,
+	0x65, 0x74, 0x69, 0x6e, 0x67, 0x22, 0x28, 0x0a, 0x12, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x74,
+	0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e,
+	0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22,
+	0x31, 0x0a, 0x13, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65,
+	0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x67, 0x72, 0x65, 0x65, 0x74, 0x69,
+	0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x67, 0x72, 0x65, 0x65, 0x74, 0x69,
+	0x6e, 0x67, 0x22, 0x2e, 0x0a, 0x18, 0x47, 0x72, 0x65, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e,
+	0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12,
+	0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61,
+	0x6d, 0x65, 0x22, 0x37, 0x0a, 0x19, 0x47, 0x72, 0x65, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e,
+	0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12,
+	0x1a, 0x0a, 0x08, 0x67, 0x72, 0x65, 0x65, 0x74, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x09, 0x52, 0x08, 0x67, 0x72, 0x65, 0x65, 0x74, 0x69, 0x6e, 0x67, 0x22, 0x2e, 0x0a, 0x18, 0x47,
+	0x72, 0x65, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
+	0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x18,
+	0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x6e, 0x61, 0x6d, 0x65, 0x22, 0x37, 0x0a, 0x19, 0x47,
+	0x72, 0x65, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d,
+	0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1a, 0x0a, 0x08, 0x67, 0x72, 0x65, 0x65,
+	0x74, 0x69, 0x6e, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x67, 0x72, 0x65, 0x65,
+	0x74, 0x69, 0x6e, 0x67, 0x32, 0xc8, 0x02, 0x0a, 0x0c, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x65,
+	0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x34, 0x0a, 0x05, 0x47, 0x72, 0x65, 0x65, 0x74, 0x12, 0x13,
+	0x2e, 0x67, 0x72, 0x65, 0x65, 0x74, 0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x52, 0x65, 0x71, 0x75,
+	0x65, 0x73, 0x74, 0x1a, 0x14, 0x2e, 0x67, 0x72, 0x65, 0x65, 0x74, 0x2e, 0x47, 0x72, 0x65, 0x65,
+	0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x4a, 0x0a, 0x0b, 0x47,
+	0x72, 0x65, 0x65, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x19, 0x2e, 0x67, 0x72, 0x65,
+	0x65, 0x74, 0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65,
+	0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1a, 0x2e, 0x67, 0x72, 0x65, 0x65, 0x74, 0x2e, 0x47, 0x72,
+	0x65, 0x65, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01, 0x12, 0x5a, 0x0a, 0x11, 0x47, 0x72, 0x65, 0x65, 0x74,
+	0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x1f, 0x2e, 0x67,
+	0x72, 0x65, 0x65, 0x74, 0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e, 0x74,
+	0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e,
+	0x67, 0x72, 0x65, 0x65, 0x74, 0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x43, 0x6c, 0x69, 0x65, 0x6e,
+	0x74, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22,
+	0x00, 0x28, 0x01, 0x12, 0x5a, 0x0a, 0x11, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76,
+	0x65, 0x72, 0x53, 0x74, 0x72, 0x65, 0x61, 0x6d, 0x12, 0x1f, 0x2e, 0x67, 0x72, 0x65, 0x65, 0x74,
+	0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x72, 0x65,
+	0x61, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x67, 0x72, 0x65, 0x65,
+	0x74, 0x2e, 0x47, 0x72, 0x65, 0x65, 0x74, 0x53, 0x65, 0x72, 0x76, 0x65, 0x72, 0x53, 0x74, 0x72,
+	0x65, 0x61, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x42,
+	0x4e, 0x5a, 0x4c, 0x64, 0x75, 0x62, 0x62, 0x6f, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e,
+	0x6f, 0x72, 0x67, 0x2f, 0x64, 0x75, 0x62, 0x62, 0x6f, 0x2d, 0x67, 0x6f, 0x2f, 0x76, 0x33, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x2f,
+	0x69, 0x6e, 0x74, 0x65, 0x72, 0x6e, 0x61, 0x6c, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x64,
+	0x75, 0x62, 0x62, 0x6f, 0x33, 0x5f, 0x67, 0x65, 0x6e, 0x3b, 0x67, 0x72, 0x65, 0x65, 0x74, 0x62,
+	0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_greet_proto_rawDescOnce sync.Once
+	file_greet_proto_rawDescData = file_greet_proto_rawDesc
+)
+
+func file_greet_proto_rawDescGZIP() []byte {
+	file_greet_proto_rawDescOnce.Do(func() {
+		file_greet_proto_rawDescData = protoimpl.X.CompressGZIP(file_greet_proto_rawDescData)
+	})
+	return file_greet_proto_rawDescData
+}
+
+var file_greet_proto_msgTypes = make([]protoimpl.MessageInfo, 8)
+var file_greet_proto_goTypes = []interface{}{
+	(*GreetRequest)(nil),              // 0: greet.GreetRequest
+	(*GreetResponse)(nil),             // 1: greet.GreetResponse
+	(*GreetStreamRequest)(nil),        // 2: greet.GreetStreamRequest
+	(*GreetStreamResponse)(nil),       // 3: greet.GreetStreamResponse
+	(*GreetClientStreamRequest)(nil),  // 4: greet.GreetClientStreamRequest
+	(*GreetClientStreamResponse)(nil), // 5: greet.GreetClientStreamResponse
+	(*GreetServerStreamRequest)(nil),  // 6: greet.GreetServerStreamRequest
+	(*GreetServerStreamResponse)(nil), // 7: greet.GreetServerStreamResponse
+}
+var file_greet_proto_depIdxs = []int32{
+	0, // 0: greet.GreetService.Greet:input_type -> greet.GreetRequest
+	2, // 1: greet.GreetService.GreetStream:input_type -> greet.GreetStreamRequest
+	4, // 2: greet.GreetService.GreetClientStream:input_type -> greet.GreetClientStreamRequest
+	6, // 3: greet.GreetService.GreetServerStream:input_type -> greet.GreetServerStreamRequest
+	1, // 4: greet.GreetService.Greet:output_type -> greet.GreetResponse
+	3, // 5: greet.GreetService.GreetStream:output_type -> greet.GreetStreamResponse
+	5, // 6: greet.GreetService.GreetClientStream:output_type -> greet.GreetClientStreamResponse
+	7, // 7: greet.GreetService.GreetServerStream:output_type -> greet.GreetServerStreamResponse
+	4, // [4:8] is the sub-list for method output_type
+	0, // [0:4] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_greet_proto_init() }
+func file_greet_proto_init() {
+	if File_greet_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_greet_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetStreamRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetStreamResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetClientStreamRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetClientStreamResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetServerStreamRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_greet_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*GreetServerStreamResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_greet_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   8,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_greet_proto_goTypes,
+		DependencyIndexes: file_greet_proto_depIdxs,
+		MessageInfos:      file_greet_proto_msgTypes,
+	}.Build()
+	File_greet_proto = out.File
+	file_greet_proto_rawDesc = nil
+	file_greet_proto_goTypes = nil
+	file_greet_proto_depIdxs = nil
+}
diff --git a/protocol/triple/internal/proto/greet.proto b/protocol/triple/internal/proto/greet.proto
new file mode 100644
index 0000000..8547d23
--- /dev/null
+++ b/protocol/triple/internal/proto/greet.proto
@@ -0,0 +1,44 @@
+syntax = "proto3";
+
+package greet;
+
+option go_package = "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/dubbo3_gen;greet";
+
+message GreetRequest {
+  string name = 1;
+}
+
+message GreetResponse {
+  string greeting = 1;
+}
+
+message GreetStreamRequest {
+  string name = 1;
+}
+
+message GreetStreamResponse {
+  string greeting = 1;
+}
+
+message GreetClientStreamRequest {
+  string name = 1;
+}
+
+message GreetClientStreamResponse {
+  string greeting = 1;
+}
+
+message GreetServerStreamRequest {
+  string name = 1;
+}
+
+message GreetServerStreamResponse {
+  string greeting = 1;
+}
+
+service GreetService {
+  rpc Greet(GreetRequest) returns (GreetResponse) {}
+  rpc GreetStream(stream GreetStreamRequest) returns (stream GreetStreamResponse) {}
+  rpc GreetClientStream(stream GreetClientStreamRequest) returns (GreetClientStreamResponse) {}
+  rpc GreetServerStream(GreetServerStreamRequest) returns (stream GreetServerStreamResponse) {}
+}
\ No newline at end of file
diff --git a/protocol/triple/internal/proto/triple_gen/greettriple/greet.triple.go b/protocol/triple/internal/proto/triple_gen/greettriple/greet.triple.go
new file mode 100644
index 0000000..69f1046
--- /dev/null
+++ b/protocol/triple/internal/proto/triple_gen/greettriple/greet.triple.go
@@ -0,0 +1,389 @@
+// Code generated by protoc-gen-triple. DO NOT EDIT.
+//
+// Source: protocol/triple/internal/proto/greet.proto
+
+package greettriple
+
+import (
+	context "context"
+	errors "errors"
+	http "net/http"
+)
+
+import (
+	client "dubbo.apache.org/dubbo-go/v3/client"
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+	triple_protocol "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+// This is a compile-time assertion to ensure that this generated file and the connect package are
+// compatible. If you get a compiler error that this constant is not defined, this code was
+// generated with a version of connect newer than the one compiled into your binary. You can fix the
+// problem by either regenerating this code with an older version of connect or updating the connect
+// version compiled into your binary.
+const _ = triple_protocol.IsAtLeastVersion0_1_0
+
+const (
+	// GreetServiceName is the fully-qualified name of the GreetService service.
+	GreetServiceName = "greet.GreetService"
+)
+
+// These constants are the fully-qualified names of the RPCs defined in this package. They're
+// exposed at runtime as Spec.Procedure and as the final two segments of the HTTP route.
+//
+// Note that these are different from the fully-qualified method names used by
+// google.golang.org/protobuf/reflect/protoreflect. To convert from these constants to
+// reflection-formatted method names, remove the leading slash and convert the remaining slash to a
+// period.
+const (
+	// GreetServiceGreetProcedure is the fully-qualified name of the GreetService's Greet RPC.
+	GreetServiceGreetProcedure = "/greet.GreetService/Greet"
+	// GreetServiceGreetStreamProcedure is the fully-qualified name of the GreetService's GreetStream
+	// RPC.
+	GreetServiceGreetStreamProcedure = "/greet.GreetService/GreetStream"
+	// GreetServiceGreetClientStreamProcedure is the fully-qualified name of the GreetService's
+	// GreetClientStream RPC.
+	GreetServiceGreetClientStreamProcedure = "/greet.GreetService/GreetClientStream"
+	// GreetServiceGreetServerStreamProcedure is the fully-qualified name of the GreetService's
+	// GreetServerStream RPC.
+	GreetServiceGreetServerStreamProcedure = "/greet.GreetService/GreetServerStream"
+)
+
+// GreetService is a client for the greet.GreetService service.
+type GreetService interface {
+	Greet(ctx context.Context, req *proto.GreetRequest, opts ...client.CallOption) (*proto.GreetResponse, error)
+	GreetStream(ctx context.Context, opts ...client.CallOption) (GreetService_GreetStreamClient, error)
+	GreetClientStream(ctx context.Context, opts ...client.CallOption) (GreetService_GreetClientStreamClient, error)
+	GreetServerStream(ctx context.Context, req *proto.GreetServerStreamRequest, opts ...client.CallOption) (GreetService_GreetServerStreamClient, error)
+}
+
+// NewGreetService constructs a client for the greet.GreetService service. By default, it uses
+// the Connect protocol with the binary Protobuf Codec, asks for gzipped responses, and sends
+// uncompressed requests. To use the gRPC or gRPC-Web protocols, supply the connect.WithGRPC() or
+// connect.WithGRPCWeb() options.
+//
+// The URL supplied here should be the base URL for the Connect or gRPC server (for example,
+// http://api.acme.com or https://acme.com/grpc).
+func NewGreetService(cli *client.Client) (GreetService, error) {
+	if err := cli.Init(&GreetService_ClientInfo); err != nil {
+		return nil, err
+	}
+	return &GreetServiceImpl{
+		cli: cli,
+	}, nil
+}
+
+func SetConsumerService(srv common.RPCService) {
+	config.SetClientInfoService(&GreetService_ClientInfo, srv)
+}
+
+// GreetServiceImpl implements GreetService.
+type GreetServiceImpl struct {
+	cli *client.Client
+}
+
+func (c *GreetServiceImpl) Greet(ctx context.Context, req *proto.GreetRequest, opts ...client.CallOption) (*proto.GreetResponse, error) {
+	resp := new(proto.GreetResponse)
+	if err := c.cli.CallUnary(ctx, req, resp, "greet.GreetService", "Greet", opts...); err != nil {
+		return nil, err
+	}
+	return resp, nil
+}
+
+func (c *GreetServiceImpl) GreetStream(ctx context.Context, opts ...client.CallOption) (GreetService_GreetStreamClient, error) {
+	stream, err := c.cli.CallBidiStream(ctx, "greet.GreetService", "GreetStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.BidiStreamForClient)
+	return &GreetServiceGreetStreamClient{rawStream}, nil
+}
+
+func (c *GreetServiceImpl) GreetClientStream(ctx context.Context, opts ...client.CallOption) (GreetService_GreetClientStreamClient, error) {
+	stream, err := c.cli.CallClientStream(ctx, "greet.GreetService", "GreetClientStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ClientStreamForClient)
+	return &GreetServiceGreetClientStreamClient{rawStream}, nil
+}
+
+func (c *GreetServiceImpl) GreetServerStream(ctx context.Context, req *proto.GreetServerStreamRequest, opts ...client.CallOption) (GreetService_GreetServerStreamClient, error) {
+	stream, err := c.cli.CallServerStream(ctx, req, "greet.GreetService", "GreetServerStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ServerStreamForClient)
+	return &GreetServiceGreetServerStreamClient{rawStream}, nil
+}
+
+type GreetService_GreetStreamClient interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.GreetStreamRequest) error
+	RequestHeader() http.Header
+	CloseRequest() error
+	Recv() (*proto.GreetStreamResponse, error)
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	CloseResponse() error
+}
+
+type GreetServiceGreetStreamClient struct {
+	*triple_protocol.BidiStreamForClient
+}
+
+func (cli *GreetServiceGreetStreamClient) Send(msg *proto.GreetStreamRequest) error {
+	return cli.BidiStreamForClient.Send(msg)
+}
+
+func (cli *GreetServiceGreetStreamClient) Recv() (*proto.GreetStreamResponse, error) {
+	msg := new(proto.GreetStreamResponse)
+	if err := cli.BidiStreamForClient.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+type GreetService_GreetClientStreamClient interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.GreetClientStreamRequest) error
+	RequestHeader() http.Header
+	CloseAndRecv() (*proto.GreetClientStreamResponse, error)
+	Conn() (triple_protocol.StreamingClientConn, error)
+}
+
+type GreetServiceGreetClientStreamClient struct {
+	*triple_protocol.ClientStreamForClient
+}
+
+func (cli *GreetServiceGreetClientStreamClient) Send(msg *proto.GreetClientStreamRequest) error {
+	return cli.ClientStreamForClient.Send(msg)
+}
+
+func (cli *GreetServiceGreetClientStreamClient) CloseAndRecv() (*proto.GreetClientStreamResponse, error) {
+	msg := new(proto.GreetClientStreamResponse)
+	resp := triple_protocol.NewResponse(msg)
+	if err := cli.ClientStreamForClient.CloseAndReceive(resp); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+func (cli *GreetServiceGreetClientStreamClient) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ClientStreamForClient.Conn()
+}
+
+type GreetService_GreetServerStreamClient interface {
+	Recv() bool
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Msg() *proto.GreetServerStreamResponse
+	Err() error
+	Conn() (triple_protocol.StreamingClientConn, error)
+	Close() error
+}
+
+type GreetServiceGreetServerStreamClient struct {
+	*triple_protocol.ServerStreamForClient
+}
+
+func (cli *GreetServiceGreetServerStreamClient) Recv() bool {
+	msg := new(proto.GreetServerStreamResponse)
+	return cli.ServerStreamForClient.Receive(msg)
+}
+
+func (cli *GreetServiceGreetServerStreamClient) Msg() *proto.GreetServerStreamResponse {
+	msg := cli.ServerStreamForClient.Msg()
+	if msg == nil {
+		return new(proto.GreetServerStreamResponse)
+	}
+	return msg.(*proto.GreetServerStreamResponse)
+}
+
+func (cli *GreetServiceGreetServerStreamClient) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ServerStreamForClient.Conn()
+}
+
+var GreetService_ClientInfo = client.ClientInfo{
+	InterfaceName: "greet.GreetService",
+	MethodNames:   []string{"Greet", "GreetStream", "GreetClientStream", "GreetServerStream"},
+	ClientInjectFunc: func(dubboCliRaw interface{}, cli *client.Client) {
+		dubboCli := dubboCliRaw.(GreetServiceImpl)
+		dubboCli.cli = cli
+	},
+}
+
+// GreetServiceHandler is an implementation of the greet.GreetService service.
+type GreetServiceHandler interface {
+	Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error)
+	GreetStream(context.Context, GreetService_GreetStreamServer) error
+	GreetClientStream(context.Context, GreetService_GreetClientStreamServer) (*proto.GreetClientStreamResponse, error)
+	GreetServerStream(context.Context, *proto.GreetServerStreamRequest, GreetService_GreetServerStreamServer) error
+}
+
+func RegisterGreetServiceHandler(srv *server.Server, hdlr GreetServiceHandler, opts ...server.ServiceOption) error {
+	return srv.Register(hdlr, &GreetService_ServiceInfo, opts...)
+}
+
+type GreetService_GreetStreamServer interface {
+	Send(*proto.GreetStreamResponse) error
+	Recv() (*proto.GreetStreamRequest, error)
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	RequestHeader() http.Header
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetStreamServer struct {
+	*triple_protocol.BidiStream
+}
+
+func (srv *greetServiceGreetStreamServer) Send(msg *proto.GreetStreamResponse) error {
+	return srv.BidiStream.Send(msg)
+}
+
+func (srv greetServiceGreetStreamServer) Recv() (*proto.GreetStreamRequest, error) {
+	msg := new(proto.GreetStreamRequest)
+	if err := srv.BidiStream.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+type GreetService_GreetClientStreamServer interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Recv() bool
+	RequestHeader() http.Header
+	Msg() *proto.GreetClientStreamRequest
+	Err() error
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetClientStreamServer struct {
+	*triple_protocol.ClientStream
+}
+
+func (srv *greetServiceGreetClientStreamServer) Recv() bool {
+	msg := new(proto.GreetClientStreamRequest)
+	return srv.ClientStream.Receive(msg)
+}
+
+func (srv *greetServiceGreetClientStreamServer) Msg() *proto.GreetClientStreamRequest {
+	msgRaw := srv.ClientStream.Msg()
+	if msgRaw == nil {
+		return new(proto.GreetClientStreamRequest)
+	}
+	return msgRaw.(*proto.GreetClientStreamRequest)
+}
+
+type GreetService_GreetServerStreamServer interface {
+	Send(*proto.GreetServerStreamResponse) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetServerStreamServer struct {
+	*triple_protocol.ServerStream
+}
+
+func (g *greetServiceGreetServerStreamServer) Send(msg *proto.GreetServerStreamResponse) error {
+	return g.ServerStream.Send(msg)
+}
+
+var GreetService_ServiceInfo = server.ServiceInfo{
+	InterfaceName: "greet.GreetService",
+	ServiceType:   (*GreetServiceHandler)(nil),
+	Methods: []server.MethodInfo{
+		{
+			Name: "Greet",
+			Type: constant.CallUnary,
+			ReqInitFunc: func() interface{} {
+				return new(proto.GreetRequest)
+			},
+			MethodFunc: func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.GreetRequest)
+				res, err := handler.(GreetServiceHandler).Greet(ctx, req)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},
+		{
+			Name: "GreetStream",
+			Type: constant.CallBidiStream,
+			StreamInitFunc: func(baseStream interface{}) interface{} {
+				return &greetServiceGreetStreamServer{baseStream.(*triple_protocol.BidiStream)}
+			},
+			MethodFunc: func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].(GreetService_GreetStreamServer)
+				if err := handler.(GreetServiceHandler).GreetStream(ctx, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},
+		{
+			Name: "GreetClientStream",
+			Type: constant.CallClientStream,
+			StreamInitFunc: func(baseStream interface{}) interface{} {
+				return &greetServiceGreetClientStreamServer{baseStream.(*triple_protocol.ClientStream)}
+			},
+			MethodFunc: func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].(GreetService_GreetClientStreamServer)
+				res, err := handler.(GreetServiceHandler).GreetClientStream(ctx, stream)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},
+		{
+			Name: "GreetServerStream",
+			Type: constant.CallServerStream,
+			ReqInitFunc: func() interface{} {
+				return new(proto.GreetServerStreamRequest)
+			},
+			StreamInitFunc: func(baseStream interface{}) interface{} {
+				return &greetServiceGreetServerStreamServer{baseStream.(*triple_protocol.ServerStream)}
+			},
+			MethodFunc: func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.GreetServerStreamRequest)
+				stream := args[1].(GreetService_GreetServerStreamServer)
+				if err := handler.(GreetServiceHandler).GreetServerStream(ctx, req, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},
+	},
+}
+
+// UnimplementedGreetServiceHandler returns CodeUnimplemented from all methods.
+type UnimplementedGreetServiceHandler struct{}
+
+func (UnimplementedGreetServiceHandler) Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greet.GreetService.Greet is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetStream(context.Context, *triple_protocol.BidiStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greet.GreetService.GreetStream is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetClientStream(context.Context, *triple_protocol.ClientStream) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greet.GreetService.GreetClientStream is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetServerStream(context.Context, *triple_protocol.Request, *triple_protocol.ServerStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greet.GreetService.GreetServerStream is not implemented"))
+}
diff --git a/protocol/triple/internal/server/api/greet_service.go b/protocol/triple/internal/server/api/greet_service.go
new file mode 100644
index 0000000..99d6818
--- /dev/null
+++ b/protocol/triple/internal/server/api/greet_service.go
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package api
+
+import (
+	"context"
+	"fmt"
+	"strings"
+)
+
+import (
+	greet "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+)
+
+type GreetTripleServer struct {
+}
+
+func (srv *GreetTripleServer) Greet(ctx context.Context, req *greet.GreetRequest) (*greet.GreetResponse, error) {
+	resp := &greet.GreetResponse{Greeting: req.Name}
+	return resp, nil
+}
+
+func (srv *GreetTripleServer) GreetStream(ctx context.Context, stream greettriple.GreetService_GreetStreamServer) error {
+	for {
+		req, err := stream.Recv()
+		if err != nil {
+			if triple.IsEnded(err) {
+				break
+			}
+			return fmt.Errorf("triple BidiStream recv error: %s", err)
+		}
+		if err := stream.Send(&greet.GreetStreamResponse{Greeting: req.Name}); err != nil {
+			return fmt.Errorf("triple BidiStream send error: %s", err)
+		}
+	}
+	return nil
+}
+
+func (srv *GreetTripleServer) GreetClientStream(ctx context.Context, stream greettriple.GreetService_GreetClientStreamServer) (*greet.GreetClientStreamResponse, error) {
+	var reqs []string
+	for stream.Recv() {
+		reqs = append(reqs, stream.Msg().Name)
+	}
+	if stream.Err() != nil && !triple.IsEnded(stream.Err()) {
+		return nil, fmt.Errorf("triple ClientStream recv err: %s", stream.Err())
+	}
+	resp := &greet.GreetClientStreamResponse{
+		Greeting: strings.Join(reqs, ","),
+	}
+
+	return resp, nil
+}
+
+func (srv *GreetTripleServer) GreetServerStream(ctx context.Context, req *greet.GreetServerStreamRequest, stream greettriple.GreetService_GreetServerStreamServer) error {
+	for i := 0; i < 5; i++ {
+		if err := stream.Send(&greet.GreetServerStreamResponse{Greeting: req.Name}); err != nil {
+			return fmt.Errorf("triple ServerStream send err: %s", err)
+		}
+	}
+	return nil
+}
diff --git a/protocol/triple/internal/server/cmd_instance/main.go b/protocol/triple/internal/server/cmd_instance/main.go
new file mode 100644
index 0000000..4c0006a
--- /dev/null
+++ b/protocol/triple/internal/server/cmd_instance/main.go
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/server/api"
+)
+
+func main() {
+	// global conception
+	// configure global configurations and common modules
+	ins, err := dubbo.NewInstance(
+		dubbo.WithName("dubbo_test"),
+		dubbo.WithProtocol("tri",
+			protocol.WithTriple(),
+			protocol.WithPort(20000),
+		),
+	)
+	srv, err := ins.NewServer()
+	if err != nil {
+		panic(err)
+	}
+	if err := greettriple.RegisterGreetServiceHandler(srv, &api.GreetTripleServer{}); err != nil {
+		panic(err)
+	}
+	if err := srv.Serve(); err != nil {
+		panic(err)
+	}
+}
diff --git a/protocol/triple/internal/server/cmd_instance_with_registry/main.go b/protocol/triple/internal/server/cmd_instance_with_registry/main.go
new file mode 100644
index 0000000..070421c
--- /dev/null
+++ b/protocol/triple/internal/server/cmd_instance_with_registry/main.go
@@ -0,0 +1,53 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3"
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/server/api"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+func main() {
+	// global conception
+	// configure global configurations and common modules
+	ins, err := dubbo.NewInstance(
+		dubbo.WithName("dubbo_test"),
+		dubbo.WithRegistry("zk",
+			registry.WithZookeeper(),
+			registry.WithAddress("127.0.0.1:2181"),
+		),
+		dubbo.WithProtocol("tri",
+			protocol.WithTriple(),
+			protocol.WithPort(20000),
+		),
+	)
+	srv, err := ins.NewServer()
+	if err != nil {
+		panic(err)
+	}
+	if err := greettriple.RegisterGreetServiceHandler(srv, &api.GreetTripleServer{}); err != nil {
+		panic(err)
+	}
+	if err := srv.Serve(); err != nil {
+		panic(err)
+	}
+}
diff --git a/protocol/triple/internal/server/cmd_server/main.go b/protocol/triple/internal/server/cmd_server/main.go
new file mode 100644
index 0000000..d7c5e5b
--- /dev/null
+++ b/protocol/triple/internal/server/cmd_server/main.go
@@ -0,0 +1,44 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/server/api"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+func main() {
+	srv, err := server.NewServer(
+		server.WithServer_Protocol("tri",
+			protocol.WithTriple(),
+			protocol.WithPort(20000),
+		),
+	)
+	if err != nil {
+		panic(err)
+	}
+	if err := greettriple.RegisterGreetServiceHandler(srv, &api.GreetTripleServer{}); err != nil {
+		panic(err)
+	}
+	if err := srv.Serve(); err != nil {
+		panic(err)
+	}
+}
diff --git a/protocol/triple/internal/server/cmd_server_with_registry/main.go b/protocol/triple/internal/server/cmd_server_with_registry/main.go
new file mode 100644
index 0000000..de2cc46
--- /dev/null
+++ b/protocol/triple/internal/server/cmd_server_with_registry/main.go
@@ -0,0 +1,49 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	_ "dubbo.apache.org/dubbo-go/v3/imports"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/server/api"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+func main() {
+	srv, err := server.NewServer(
+		server.WithServer_Registry("zk",
+			registry.WithZookeeper(),
+			registry.WithAddress("127.0.0.1:2181"),
+		),
+		server.WithServer_Protocol("tri",
+			protocol.WithTriple(),
+			protocol.WithPort(20000),
+		),
+	)
+	if err != nil {
+		panic(err)
+	}
+	if err := greettriple.RegisterGreetServiceHandler(srv, &api.GreetTripleServer{}); err != nil {
+		panic(err)
+	}
+	if err := srv.Serve(); err != nil {
+		panic(err)
+	}
+}
diff --git a/protocol/triple/server.go b/protocol/triple/server.go
new file mode 100644
index 0000000..64ece57
--- /dev/null
+++ b/protocol/triple/server.go
@@ -0,0 +1,319 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"context"
+	"crypto/tls"
+	"fmt"
+	"net/http"
+	"sync"
+	"time"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+
+	"github.com/dustin/go-humanize"
+
+	"golang.org/x/net/http2"
+	"golang.org/x/net/http2/h2c"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/dubbo3"
+	"dubbo.apache.org/dubbo-go/v3/protocol/invocation"
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+// Server is TRIPLE server
+type Server struct {
+	httpServer *http.Server
+}
+
+// NewServer creates a new TRIPLE server
+func NewServer() *Server {
+	return &Server{}
+}
+
+// Start TRIPLE server
+func (s *Server) Start(invoker protocol.Invoker, info *server.ServiceInfo) {
+	var (
+		addr    string
+		err     error
+		URL     *common.URL
+		hanOpts []tri.HandlerOption
+	)
+	URL = invoker.GetURL()
+	addr = URL.Location
+	srv := &http.Server{
+		Addr: addr,
+	}
+
+	maxServerRecvMsgSize := constant.DefaultMaxServerRecvMsgSize
+	if recvMsgSize, convertErr := humanize.ParseBytes(URL.GetParam(constant.MaxServerRecvMsgSize, "")); convertErr == nil && recvMsgSize != 0 {
+		maxServerRecvMsgSize = int(recvMsgSize)
+	}
+	hanOpts = append(hanOpts, tri.WithReadMaxBytes(maxServerRecvMsgSize))
+
+	maxServerSendMsgSize := constant.DefaultMaxServerSendMsgSize
+	if sendMsgSize, convertErr := humanize.ParseBytes(URL.GetParam(constant.MaxServerSendMsgSize, "")); err == convertErr && sendMsgSize != 0 {
+		maxServerSendMsgSize = int(sendMsgSize)
+	}
+	hanOpts = append(hanOpts, tri.WithSendMaxBytes(maxServerSendMsgSize))
+
+	serialization := URL.GetParam(constant.SerializationKey, constant.ProtobufSerialization)
+	switch serialization {
+	case constant.ProtobufSerialization:
+	case constant.JSONSerialization:
+	default:
+		panic(fmt.Sprintf("Unsupported serialization: %s", serialization))
+	}
+
+	// todo: implement interceptor
+	// If global trace instance was set, then server tracer instance
+	// can be get. If not, will return NoopTracer.
+	//tracer := opentracing.GlobalTracer()
+	//serverOpts = append(serverOpts,
+	//	grpc.UnaryInterceptor(otgrpc.OpenTracingServerInterceptor(tracer)),
+	//	grpc.StreamInterceptor(otgrpc.OpenTracingStreamServerInterceptor(tracer)),
+	//	grpc.MaxRecvMsgSize(maxServerRecvMsgSize),
+	//	grpc.MaxSendMsgSize(maxServerSendMsgSize),
+	//)
+	var cfg *tls.Config
+	// todo(DMwangnima): think about a more elegant way to configure tls
+	//tlsConfig := config.GetRootConfig().TLSConfig
+	//if tlsConfig != nil {
+	//	cfg, err = config.GetServerTlsConfig(&config.TLSConfig{
+	//		CACertFile:    tlsConfig.CACertFile,
+	//		TLSCertFile:   tlsConfig.TLSCertFile,
+	//		TLSKeyFile:    tlsConfig.TLSKeyFile,
+	//		TLSServerName: tlsConfig.TLSServerName,
+	//	})
+	//	if err != nil {
+	//		return
+	//	}
+	//	logger.Infof("Triple Server initialized the TLSConfig configuration")
+	//}
+	//srv.TLSConfig = cfg
+
+	// todo:// open tracing
+	hanOpts = append(hanOpts, tri.WithInterceptors())
+	// todo:// move tls config to handleService
+	s.httpServer = srv
+
+	go func() {
+		mux := http.NewServeMux()
+		if info != nil {
+			handleServiceWithInfo(invoker, info, mux)
+		} else {
+			compatHandleService(mux)
+		}
+		// todo: figure it out this process
+		//reflection.Register(server)
+		// todo: without tls
+		if cfg == nil {
+			srv.Handler = h2c.NewHandler(mux, &http2.Server{})
+		} else {
+			srv.Handler = mux
+		}
+
+		if err = srv.ListenAndServe(); err != nil {
+			logger.Errorf("server serve failed with err: %v", err)
+		}
+	}()
+}
+
+// getSyncMapLen gets sync map len
+func getSyncMapLen(m *sync.Map) int {
+	length := 0
+
+	m.Range(func(_, _ interface{}) bool {
+		length++
+		return true
+	})
+	return length
+}
+
+// waitTripleExporter wait until len(providerServices) = len(ExporterMap)
+func waitTripleExporter(providerServices map[string]*config.ServiceConfig) {
+	t := time.NewTicker(50 * time.Millisecond)
+	defer t.Stop()
+	pLen := len(providerServices)
+	ta := time.NewTimer(10 * time.Second)
+	defer ta.Stop()
+
+	for {
+		select {
+		case <-t.C:
+			mLen := getSyncMapLen(tripleProtocol.ExporterMap())
+			if pLen == mLen {
+				return
+			}
+		case <-ta.C:
+			panic("wait Triple exporter timeout when start GRPC_NEW server")
+		}
+	}
+}
+
+// *Important*, this function is responsible for being compatible with old triple-gen code
+// compatHandleService creates handler based on ServiceConfig and provider service.
+func compatHandleService(mux *http.ServeMux, opts ...tri.HandlerOption) {
+	providerServices := config.GetProviderConfig().Services
+	if len(providerServices) == 0 {
+		panic("Provider service map is null")
+	}
+	//waitTripleExporter(providerServices)
+	for key, providerService := range providerServices {
+		// todo(DMwangnima): judge protocol type
+		service := config.GetProviderService(key)
+		ds, ok := service.(dubbo3.Dubbo3GrpcService)
+		if !ok {
+			panic("illegal service type registered")
+		}
+
+		serviceKey := common.ServiceKey(providerService.Interface, providerService.Group, providerService.Version)
+		exporter, _ := tripleProtocol.ExporterMap().Load(serviceKey)
+		if exporter == nil {
+			// todo(DMwangnima): handler reflection Service and health Service
+			continue
+			//panic(fmt.Sprintf("no exporter found for servicekey: %v", serviceKey))
+		}
+		invoker := exporter.(protocol.Exporter).GetInvoker()
+		if invoker == nil {
+			panic(fmt.Sprintf("no invoker found for servicekey: %v", serviceKey))
+		}
+
+		// inject invoker, it has all invocation logics
+		ds.XXX_SetProxyImpl(invoker)
+		path, handler := compatBuildHandler(ds, opts...)
+		mux.Handle(path, handler)
+	}
+}
+
+func compatBuildHandler(svc dubbo3.Dubbo3GrpcService, opts ...tri.HandlerOption) (string, http.Handler) {
+	mux := http.NewServeMux()
+	desc := svc.XXX_ServiceDesc()
+	basePath := desc.ServiceName
+	// init unary handlers
+	for _, method := range desc.Methods {
+		// please refer to protocol/triple/internal/proto/triple_gen/greettriple for procedure examples
+		// error could be ignored because base is empty string
+		procedure := joinProcedure(desc.ServiceName, method.MethodName)
+		handler := tri.NewCompatUnaryHandler(procedure, svc, tri.MethodHandler(method.Handler), opts...)
+		mux.Handle(procedure, handler)
+	}
+
+	// init stream handlers
+	for _, stream := range desc.Streams {
+		// please refer to protocol/triple/internal/proto/triple_gen/greettriple for procedure examples
+		// error could be ignored because base is empty string
+		procedure := joinProcedure(desc.ServiceName, stream.StreamName)
+		var typ tri.StreamType
+		switch {
+		case stream.ClientStreams && stream.ServerStreams:
+			typ = tri.StreamTypeBidi
+		case stream.ClientStreams:
+			typ = tri.StreamTypeClient
+		case stream.ServerStreams:
+			typ = tri.StreamTypeServer
+		}
+		handler := tri.NewCompatStreamHandler(procedure, svc, typ, stream.Handler, opts...)
+		mux.Handle(procedure, handler)
+	}
+
+	return "/" + basePath + "/", mux
+}
+
+// handleServiceWithInfo injects invoker and create handler based on ServiceInfo
+func handleServiceWithInfo(invoker protocol.Invoker, info *server.ServiceInfo, mux *http.ServeMux, opts ...tri.HandlerOption) {
+	for _, method := range info.Methods {
+		m := method
+		var handler http.Handler
+		procedure := joinProcedure(info.InterfaceName, method.Name)
+		switch m.Type {
+		case constant.CallUnary:
+			handler = tri.NewUnaryHandler(
+				procedure,
+				m.ReqInitFunc,
+				func(ctx context.Context, req *tri.Request) (*tri.Response, error) {
+					var args []interface{}
+					args = append(args, req.Msg)
+					// todo: inject method.Meta to attachments
+					invo := invocation.NewRPCInvocation(m.Name, args, nil)
+					res := invoker.Invoke(ctx, invo)
+					// todo(DMwangnima): if we do not use MethodInfo.MethodFunc, create Response manually
+					return res.Result().(*tri.Response), res.Error()
+				},
+				opts...,
+			)
+		case constant.CallClientStream:
+			handler = tri.NewClientStreamHandler(
+				procedure,
+				func(ctx context.Context, stream *tri.ClientStream) (*tri.Response, error) {
+					var args []interface{}
+					args = append(args, m.StreamInitFunc(stream))
+					invo := invocation.NewRPCInvocation(m.Name, args, nil)
+					res := invoker.Invoke(ctx, invo)
+					return res.Result().(*tri.Response), res.Error()
+				},
+			)
+		case constant.CallServerStream:
+			handler = tri.NewServerStreamHandler(
+				procedure,
+				m.ReqInitFunc,
+				func(ctx context.Context, request *tri.Request, stream *tri.ServerStream) error {
+					var args []interface{}
+					args = append(args, request.Msg, m.StreamInitFunc(stream))
+					invo := invocation.NewRPCInvocation(m.Name, args, nil)
+					res := invoker.Invoke(ctx, invo)
+					return res.Error()
+				},
+			)
+		case constant.CallBidiStream:
+			handler = tri.NewBidiStreamHandler(
+				procedure,
+				func(ctx context.Context, stream *tri.BidiStream) error {
+					var args []interface{}
+					args = append(args, m.StreamInitFunc(stream))
+					invo := invocation.NewRPCInvocation(m.Name, args, nil)
+					res := invoker.Invoke(ctx, invo)
+					return res.Error()
+				},
+			)
+		}
+		mux.Handle(procedure, handler)
+	}
+}
+
+// Stop TRIPLE server
+func (s *Server) Stop() {
+	// todo: process error
+	s.httpServer.Close()
+}
+
+// GracefulStop TRIPLE server
+func (s *Server) GracefulStop() {
+	// todo: process error and use timeout
+	s.httpServer.Shutdown(context.Background())
+}
diff --git a/protocol/triple/triple-tool/LICENSE b/protocol/triple/triple-tool/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/protocol/triple/triple-tool/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed under the Apache License, Version 2.0 (the "License");
+   you may not use this file except in compliance with the License.
+   You may obtain a copy of the License at
+
+       http://www.apache.org/licenses/LICENSE-2.0
+
+   Unless required by applicable law or agreed to in writing, software
+   distributed under the License is distributed on an "AS IS" BASIS,
+   WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+   See the License for the specific language governing permissions and
+   limitations under the License.
diff --git a/protocol/triple/triple-tool/cmd/root.go b/protocol/triple/triple-tool/cmd/root.go
new file mode 100644
index 0000000..9698dd7
--- /dev/null
+++ b/protocol/triple/triple-tool/cmd/root.go
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package cmd
+
+import (
+	"os"
+)
+
+import (
+	"github.com/spf13/cobra"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/gen"
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/internal/version"
+)
+
+// rootCmd represents the base command when called without any subcommands
+var rootCmd = &cobra.Command{
+	Use: "triple-tool",
+}
+
+// Execute adds all child commands to the root command and sets flags appropriately.
+// This is called by main.main(). It only needs to happen once to the rootCmd.
+func Execute() {
+	err := rootCmd.Execute()
+	if err != nil {
+		os.Exit(1)
+	}
+}
+
+func init() {
+	rootCmd.Version = version.Version
+
+	rootCmd.AddCommand(gen.Cmd)
+}
diff --git a/protocol/triple/triple-tool/gen/cmd.go b/protocol/triple/triple-tool/gen/cmd.go
new file mode 100644
index 0000000..ae9b0c2
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/cmd.go
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package gen
+
+import (
+	"github.com/spf13/cobra"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/gen/generator"
+)
+
+var Cmd = &cobra.Command{
+	Use:  "gen",
+	RunE: generator.Generate,
+}
+
+func init() {
+	flags := Cmd.Flags()
+
+	flags.StringVar(&generator.ProtocPath, "protoPath", ".", "")
+	flags.StringSliceVar(&generator.GoOpts, "go_opt", []string{}, "")
+	flags.MarkHidden("protoPath")
+	flags.MarkHidden("go_opt")
+}
diff --git a/protocol/triple/triple-tool/gen/generator/args.go b/protocol/triple/triple-tool/gen/generator/args.go
new file mode 100644
index 0000000..3ab99d5
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/args.go
@@ -0,0 +1,23 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+var (
+	ProtocPath string
+	GoOpts     []string
+)
diff --git a/protocol/triple/triple-tool/gen/generator/context.go b/protocol/triple/triple-tool/gen/generator/context.go
new file mode 100644
index 0000000..aae84ee
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/context.go
@@ -0,0 +1,87 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+import (
+	"os"
+	"path/filepath"
+)
+
+import (
+	"github.com/spf13/cobra"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/util"
+)
+
+type Context struct {
+	Src          string
+	ProtocCmd    string
+	GoOpts       []string
+	GoOut        string
+	GoModuleName string
+	Pwd          string
+}
+
+func newContext(cmd *cobra.Command, args []string) (Context, error) {
+	var ctx Context
+	pwd, err := os.Getwd()
+	if err != nil {
+		return ctx, err
+	}
+	ctx.Pwd = pwd
+	src, err := filepath.Abs(ProtocPath)
+	if err != nil {
+		return ctx, err
+	}
+	ctx.Src = src
+	ctx.GoOut = filepath.Dir(src)
+	module, err := util.GetModuleName()
+	if err != nil {
+		return ctx, err
+	}
+	ctx.GoModuleName = module
+	ctx.GoOpts = GoOpts
+	return ctx, nil
+}
+
+func NewContextForProtoc(src string) (Context, error) {
+	var ctx Context
+	ctx.Src = src
+	moduleName, err := util.GetModuleName()
+	if err != nil {
+		return ctx, err
+	}
+	ctx.GoModuleName = moduleName
+	pwd, err := os.Getwd()
+	if err != nil {
+		return ctx, err
+	}
+	ctx.Pwd = pwd
+	ctx.Src = src
+	return ctx, nil
+}
+
+func Generate(cmd *cobra.Command, args []string) error {
+	ctx, err := newContext(cmd, args)
+	if err != nil {
+		return err
+	}
+	return NewGenerator(ctx).gen()
+}
diff --git a/protocol/triple/triple-tool/gen/generator/gen.go b/protocol/triple/triple-tool/gen/generator/gen.go
new file mode 100644
index 0000000..6644dfc
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/gen.go
@@ -0,0 +1,41 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+type Generator struct {
+	ctx Context
+}
+
+func NewGenerator(ctx Context) *Generator {
+	return &Generator{ctx: ctx}
+}
+
+func (g *Generator) gen() error {
+
+	err := g.GenPb()
+	if err != nil {
+		return err
+	}
+
+	err = g.GenTriple()
+	if err != nil {
+		return err
+	}
+
+	return err
+}
diff --git a/protocol/triple/triple-tool/gen/generator/genPb.go b/protocol/triple/triple-tool/gen/generator/genPb.go
new file mode 100644
index 0000000..f993eb1
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/genPb.go
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+import (
+	"fmt"
+	"log"
+	"os"
+	"path/filepath"
+	"regexp"
+	"strings"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/util"
+)
+
+func (g *Generator) GenPb() error {
+	pwd := g.ctx.Pwd
+	g.genPbCmd(pwd)
+	output, err := util.Exec(g.ctx.ProtocCmd, pwd)
+	if len(output) > 0 {
+		fmt.Println(output)
+	}
+	return err
+}
+
+func (g *Generator) genPbCmd(goout string) {
+	src := g.ctx.Src
+	g.ctx.ProtocCmd = fmt.Sprintf("protoc %s -I=%s --go_out=%s", filepath.Base(src), filepath.Dir(src), goout)
+
+	// Check if we need to add the --go_opt=module parameter
+	if g.isGoPackageFull() {
+		g.ctx.ProtocCmd += fmt.Sprintf(" --go_opt=module=%s", g.ctx.GoModuleName)
+	}
+
+	if len(g.ctx.GoOpts) > 0 {
+		g.ctx.ProtocCmd += " --go_opt=" + strings.Join(g.ctx.GoOpts, ",")
+	}
+}
+
+// Check if go_package is a full package path
+func (g *Generator) isGoPackageFull() bool {
+	// Parse go_package from the proto file
+	goPackage := g.parseGoPackageFromProto()
+
+	// Check if goPackage is a full package path, e.g., starts with "github.com/"
+	return strings.HasPrefix(goPackage, g.ctx.GoModuleName)
+}
+
+// Parse go_package from the proto file
+func (g *Generator) parseGoPackageFromProto() string {
+	protoFileContent := g.loadProtoFileContent() // Load the content of the proto file
+
+	// Use regular expression to match go_package in the proto file
+	goPackageRegex := regexp.MustCompile(`go_package\s*=\s*"([^"]+)"`)
+	match := goPackageRegex.FindStringSubmatch(protoFileContent)
+
+	if len(match) > 1 {
+		return match[1] // Return the matched go_package
+	}
+
+	return "" // Return an empty string if go_package is not matched
+}
+
+// Load the content of the proto file
+func (g *Generator) loadProtoFileContent() string {
+	protoFilePath := g.ctx.Src
+	content, err := os.ReadFile(protoFilePath)
+	if err != nil {
+		log.Fatalf("Failed to read proto file: %v", err)
+	}
+	return string(content)
+}
diff --git a/protocol/triple/triple-tool/gen/generator/genTriple.go b/protocol/triple/triple-tool/gen/generator/genTriple.go
new file mode 100644
index 0000000..246549b
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/genTriple.go
@@ -0,0 +1,224 @@
+/*
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements.  See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License.  You may obtain a copy of the License at
+*
+*     http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+ */
+
+package generator
+
+import (
+	"os"
+	"path/filepath"
+	"strings"
+)
+
+import (
+	"github.com/emicklei/proto"
+
+	"github.com/golang/protobuf/protoc-gen-go/descriptor"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/util"
+)
+
+func (g *Generator) GenTriple() error {
+	p, err := g.parseFileToProto(g.ctx.Src)
+	if err != nil {
+		return err
+	}
+	triple, err := g.parseProtoToTriple(p)
+	if err != nil {
+		return err
+	}
+	basePath, err := os.Getwd()
+	if err != nil {
+		return err
+	}
+	triple.Source, err = filepath.Rel(basePath, g.ctx.Src)
+	if err != nil {
+		return err
+	}
+	data, err := g.parseTripleToString(triple)
+	if err != nil {
+		return err
+	}
+	g.parseGOout(triple)
+	return g.generateToFile(g.ctx.GoOut, []byte(data))
+}
+
+func (g *Generator) parseFileToProto(filePath string) (*proto.Proto, error) {
+	file, err := os.Open(filePath)
+	if err != nil {
+		return nil, err
+	}
+	defer file.Close()
+
+	parser := proto.NewParser(file)
+	p, err := parser.Parse()
+	if err != nil {
+		return nil, err
+	}
+	return p, nil
+}
+
+func (g *Generator) parseProtoToTriple(p *proto.Proto) (TripleGo, error) {
+	var tripleGo TripleGo
+	proto.Walk(
+		p,
+		proto.WithPackage(func(p *proto.Package) {
+			tripleGo.ProtoPackage = p.Name
+			tripleGo.Package = p.Name + "triple"
+		}),
+		proto.WithService(func(p *proto.Service) {
+			s := Service{ServiceName: p.Name}
+			for _, visitee := range p.Elements {
+				if vi, ok := visitee.(*proto.RPC); ok {
+					md := Method{
+						MethodName:     vi.Name,
+						RequestType:    vi.RequestType,
+						StreamsRequest: vi.StreamsRequest,
+						ReturnType:     vi.ReturnsType,
+						StreamsReturn:  vi.StreamsReturns,
+					}
+					s.Methods = append(s.Methods, md)
+				}
+			}
+			tripleGo.Services = append(tripleGo.Services, s)
+		}),
+		proto.WithOption(func(p *proto.Option) {
+			if p.Name == "go_package" {
+				i := p.Constant.Source
+				i = strings.Trim(i, "/")
+				if strings.Contains(i, g.ctx.GoModuleName) {
+					tripleGo.Import = strings.Split(i, ";")[0]
+				} else {
+					tripleGo.Import = g.ctx.GoModuleName + "/" + strings.Split(i, ";")[0]
+				}
+			}
+		}),
+	)
+	return tripleGo, nil
+}
+
+func (g *Generator) parseTripleToString(t TripleGo) (string, error) {
+	var builder strings.Builder
+
+	for _, tpl := range Tpls {
+		err := tpl.Execute(&builder, t)
+		if err != nil {
+			return "", err
+		}
+	}
+
+	return builder.String(), nil
+}
+
+func (g *Generator) parseGOout(triple TripleGo) {
+	prefix := strings.TrimPrefix(triple.Import, g.ctx.GoModuleName)
+	g.ctx.GoOut = filepath.Join(g.ctx.Pwd, filepath.Join(prefix, triple.Package+"/"+triple.ProtoPackage+".triple.go"))
+}
+
+func (g *Generator) generateToFile(filePath string, data []byte) error {
+	err := os.MkdirAll(filepath.Dir(filePath), os.ModePerm)
+	if err != nil {
+		return err
+	}
+	return os.WriteFile(filePath, data, 0666)
+}
+
+func ProcessProtoFile(file *descriptor.FileDescriptorProto) (TripleGo, error) {
+	tripleGo := TripleGo{
+		Source:       file.GetName(),
+		Package:      file.GetPackage() + "triple",
+		ProtoPackage: file.GetPackage(),
+		Services:     make([]Service, 0),
+	}
+
+	for _, service := range file.GetService() {
+		serviceMethods := make([]Method, 0)
+
+		for _, method := range service.GetMethod() {
+			serviceMethods = append(serviceMethods, Method{
+				MethodName:     method.GetName(),
+				RequestType:    strings.Split(method.GetInputType(), ".")[len(strings.Split(method.GetInputType(), "."))-1],
+				StreamsRequest: method.GetClientStreaming(),
+				ReturnType:     strings.Split(method.GetOutputType(), ".")[len(strings.Split(method.GetOutputType(), "."))-1],
+				StreamsReturn:  method.GetServerStreaming(),
+			})
+		}
+
+		tripleGo.Services = append(tripleGo.Services, Service{
+			ServiceName: service.GetName(),
+			Methods:     serviceMethods,
+		})
+	}
+
+	goPkg := file.Options.GetGoPackage()
+	goPkg = strings.Split(goPkg, ";")[0]
+	goPkg = strings.Trim(goPkg, "/")
+	moduleName, err := util.GetModuleName()
+	if err != nil {
+		return tripleGo, err
+	}
+
+	if strings.Contains(goPkg, moduleName) {
+		tripleGo.Import = strings.Split(goPkg, ";")[0]
+	} else {
+		tripleGo.Import = moduleName + "/" + strings.Split(goPkg, ";")[0]
+	}
+
+	return tripleGo, nil
+}
+
+func GenTripleFile(triple TripleGo) error {
+	module, err := util.GetModuleName()
+	if err != nil {
+		return err
+	}
+	prefix := strings.TrimPrefix(triple.Import, module)
+	pwd, err := os.Getwd()
+	if err != nil {
+		return err
+	}
+	GoOut := filepath.Join(pwd, filepath.Join(prefix, triple.Package+"/"+triple.ProtoPackage+".triple.go"))
+
+	g := &Generator{}
+	data, err := g.parseTripleToString(triple)
+	if err != nil {
+		return err
+	}
+	return g.generateToFile(GoOut, []byte(data))
+}
+
+type TripleGo struct {
+	Source       string
+	Package      string
+	Import       string
+	ProtoPackage string
+	Services     []Service
+}
+
+type Service struct {
+	ServiceName string
+	Methods     []Method
+}
+
+type Method struct {
+	MethodName     string
+	RequestType    string
+	StreamsRequest bool
+	ReturnType     string
+	StreamsReturn  bool
+}
diff --git a/protocol/triple/triple-tool/gen/generator/tripleTpl.go b/protocol/triple/triple-tool/gen/generator/tripleTpl.go
new file mode 100644
index 0000000..7dbf7ab
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/tripleTpl.go
@@ -0,0 +1,537 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+import (
+	"html/template"
+	"log"
+	"strings"
+)
+
+var (
+	Tpls                   []*template.Template
+	TplPreamble            *template.Template
+	TplPackage             *template.Template
+	TplImport              *template.Template
+	TplTotal               *template.Template
+	TplClientInterface     *template.Template
+	TplClientInterfaceImpl *template.Template
+	TplClientImpl          *template.Template
+	TplMethodInfo          *template.Template
+	TplHandler             *template.Template
+	TplServerImpl          *template.Template
+	TplServerInfo          *template.Template
+	TplUnImpl              *template.Template
+)
+
+func init() {
+	var err error
+	TplPreamble, err = template.New("preamble").Parse(PreambleTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplPackage, err = template.New("package").Parse(PackageTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplImport, err = template.New("import").Parse(ImportTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplTotal, err = template.New("total").Parse(TotalTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplClientInterface, err = template.New("clientInterface").Parse(ClientInterfaceTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplClientInterfaceImpl, err = template.New("clientInterfaceImpl").Funcs(template.FuncMap{
+		"lower": func(s string) string {
+			if s == "" {
+				return ""
+			}
+			return strings.ToLower(s[:1]) + s[1:]
+		},
+	}).Parse(ClientInterfaceImplTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplClientImpl, err = template.New("clientImpl").Funcs(template.FuncMap{
+		"lower": func(s string) string {
+			if s == "" {
+				return ""
+			}
+			return strings.ToLower(s[:1]) + s[1:]
+		},
+	}).Parse(ClientImplTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplMethodInfo, err = template.New("methodInfo").Funcs(template.FuncMap{
+		"last": func(index, length int) bool {
+			return index == length-1
+		},
+	}).Parse(MethodInfoTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplHandler, err = template.New("handler").Parse(HandlerTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplServerImpl, err = template.New("serverImpl").Funcs(template.FuncMap{
+		"lower": func(s string) string {
+			if s == "" {
+				return ""
+			}
+			return strings.ToLower(s[:1]) + s[1:]
+		},
+	}).Parse(ServerImplTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplServerInfo, err = template.New("serverInfo").Funcs(template.FuncMap{
+		"lower": func(s string) string {
+			if s == "" {
+				return ""
+			}
+			return strings.ToLower(s[:1]) + s[1:]
+		},
+	}).Parse(ServiceInfoTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+	TplUnImpl, err = template.New("unImpl").Parse(UnImplServiceTpl)
+	if err != nil {
+		log.Fatal(err)
+	}
+
+	Tpls = append(Tpls, TplPreamble)
+	Tpls = append(Tpls, TplPackage)
+	Tpls = append(Tpls, TplImport)
+	Tpls = append(Tpls, TplTotal)
+	Tpls = append(Tpls, TplClientInterface)
+	Tpls = append(Tpls, TplClientInterfaceImpl)
+	Tpls = append(Tpls, TplClientImpl)
+	Tpls = append(Tpls, TplMethodInfo)
+	Tpls = append(Tpls, TplHandler)
+	Tpls = append(Tpls, TplServerImpl)
+	Tpls = append(Tpls, TplServerInfo)
+	Tpls = append(Tpls, TplUnImpl)
+}
+
+const PreambleTpl = `// Code generated by protoc-gen-triple. DO NOT EDIT.
+//
+// Source: {{.Source}}
+`
+
+const PackageTpl = `package {{.Package}}`
+
+const ImportTpl = `
+
+import (
+	context "context"
+	errors "errors"
+	http "net/http"
+
+	client "dubbo.apache.org/dubbo-go/v3/client"
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	proto "{{.Import}}"
+	triple_protocol "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/provider"
+)
+
+`
+
+const TotalTpl = `// This is a compile-time assertion to ensure that this generated file and the connect package are
+// compatible. If you get a compiler error that this constant is not defined, this code was
+// generated with a version of connect newer than the one compiled into your binary. You can fix the
+// problem by either regenerating this code with an older version of connect or updating the connect
+// version compiled into your binary.
+const _ = triple_protocol.IsAtLeastVersion0_1_0
+{{$t := .}}{{range $s := .Services}}
+const (
+	// {{$s.ServiceName}}Name is the fully-qualified name of the {{$s.ServiceName}} service.
+	{{$s.ServiceName}}Name = "{{$t.ProtoPackage}}.{{$s.ServiceName}}"
+)
+
+// These constants are the fully-qualified names of the RPCs defined in this package. They're
+// exposed at runtime as Spec.Procedure and as the final two segments of the HTTP route.
+//
+// Note that these are different from the fully-qualified method names used by
+// google.golang.org/protobuf/reflect/protoreflect. To convert from these constants to
+// reflection-formatted method names, remove the leading slash and convert the remaining slash to a
+// period.
+const (
+{{range $s.Methods}}	// {{$s.ServiceName}}{{.MethodName}}Procedure is the fully-qualified name of the {{$s.ServiceName}}'s {{.MethodName}} RPC.
+	{{$s.ServiceName}}{{.MethodName}}Procedure = "/{{$t.ProtoPackage}}.{{$s.ServiceName}}/{{.MethodName}}"
+{{end}}){{end}}
+
+`
+
+const ClientInterfaceTpl = `//{{$t := .}}{{range $s := .Services}}{{.ServiceName}}Client is a client for the {{$t.ProtoPackage}}.{{$s.ServiceName}} service.
+type {{$s.ServiceName}}Client interface {
+	{{range $s.Methods}}
+		{{.MethodName}}(ctx context.Context{{if .StreamsRequest}}{{else}}, req *proto.{{.RequestType}}{{end}}, opt ...client.CallOption) {{if or .StreamsReturn .StreamsRequest}}({{$s.ServiceName}}_{{.MethodName}}Client, error){{else}}(*proto.{{.ReturnType}}, error){{end}}
+	{{end}}
+}{{end}}
+
+`
+
+const ClientInterfaceImplTpl = `{{$t := .}}{{range $s := .Services}}// New{{.ServiceName}}Client constructs a client for the {{$t.Package}}.{{.ServiceName}} service. By default, it uses
+// the Connect protocol with the binary Protobuf Codec, asks for gzipped responses, and sends
+// uncompressed requests. To use the gRPC or gRPC-Web protocols, supply the connect.WithGRPC() or
+// connect.WithGRPCWeb() options.
+//
+// The URL supplied here should be the base URL for the Connect or gRPC server (for example,
+// http://api.acme.com or https://acme.com/grpc).
+func New{{.ServiceName}}Client(cli *client.Client) ({{.ServiceName}}Client, error) {
+	if err := cli.Init(&{{.ServiceName}}_ClientInfo); err != nil {
+		return nil, err
+	}
+	return &{{.ServiceName}}ClientImpl{
+		cli: cli,
+	}, nil
+}
+
+func SetConsumerService(srv common.RPCService) {
+	config.SetClientInfoService(&{{.ServiceName}}_ClientInfo, srv)
+}
+
+// {{.ServiceName}}ClientImpl implements {{.ServiceName}}Client.
+type {{.ServiceName}}ClientImpl struct {
+	cli *client.Client
+}
+{{range .Methods}}{{if .StreamsRequest}}{{if .StreamsReturn}}
+func (c *{{$s.ServiceName}}ClientImpl) {{.MethodName}}(ctx context.Context, opts ...client.CallOption)({{$s.ServiceName}}_{{.MethodName}}Client,error) {
+	stream, err := c.cli.CallBidiStream(ctx, "{{$t.ProtoPackage}}.{{$s.ServiceName}}", "{{.MethodName}}", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.BidiStreamForClient)
+	return &{{lower $s.ServiceName}}{{.MethodName}}Client{rawStream}, nil
+}{{else}}
+func (c *{{$s.ServiceName}}ClientImpl) {{.MethodName}}(ctx context.Context, opts ...client.CallOption) ({{$s.ServiceName}}_{{.MethodName}}Client, error) {
+	stream, err := c.cli.CallClientStream(ctx, "{{$t.ProtoPackage}}.{{$s.ServiceName}}", "{{.MethodName}}", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ClientStreamForClient)
+	return &{{lower $s.ServiceName}}{{.MethodName}}Client{rawStream}, nil
+}{{end}}{{else}}{{if .StreamsReturn}}
+func (c *{{$s.ServiceName}}ClientImpl) {{.MethodName}}(ctx context.Context, req *proto.{{.RequestType}}, opts ...client.CallOption) ({{$s.ServiceName}}_{{.MethodName}}Client, error) {
+	triReq := triple_protocol.NewRequest(req)
+	stream, err := c.cli.CallServerStream(ctx, triReq, "{{$t.ProtoPackage}}.{{$s.ServiceName}}", "{{.MethodName}}", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ServerStreamForClient)
+	return &{{lower $s.ServiceName}}{{.MethodName}}Client{rawStream}, nil
+}{{else}}
+func (c *{{$s.ServiceName}}ClientImpl) {{.MethodName}}(ctx context.Context, req *proto.{{.RequestType}}, opts ...client.CallOption) (*proto.{{.ReturnType}}, error) {
+	triReq := triple_protocol.NewRequest(req)
+	resp := new(proto.{{.ReturnType}})
+	triResp := triple_protocol.NewResponse(resp)
+	if err := c.cli.CallUnary(ctx, triReq, triResp, "{{$t.ProtoPackage}}.{{$s.ServiceName}}", "{{.MethodName}}", opts...); err != nil {
+		return nil, err
+	}
+	return resp, nil
+}{{end}}{{end}}
+{{end}}{{end}}
+`
+
+const ClientImplTpl = `{{$t := .}}{{range $s := .Services}}{{range .Methods}}{{if .StreamsRequest}}{{if .StreamsReturn}} 
+type {{$s.ServiceName}}_{{.MethodName}}Client interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.{{.RequestType}}) error
+	RequestHeader() http.Header
+	CloseRequest() error
+	Recv() (*proto.{{.ReturnType}}, error)
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	CloseResponse() error
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Client struct {
+	*triple_protocol.BidiStreamForClient
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Send(msg *proto.{{.RequestType}}) error {
+	return cli.BidiStreamForClient.Send(msg)
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Recv() (*proto.{{.ReturnType}}, error) {
+	msg := new(proto.{{.ReturnType}})
+	if err := cli.BidiStreamForClient.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+{{else}}
+type {{$s.ServiceName}}_{{.MethodName}}Client interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.{{.RequestType}}) error
+	RequestHeader() http.Header
+	CloseAndRecv() (*proto.{{.ReturnType}}, error)
+	Conn() (triple_protocol.StreamingClientConn, error)
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Client struct {
+	*triple_protocol.ClientStreamForClient
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Send(msg *proto.{{.RequestType}}) error {
+	return cli.ClientStreamForClient.Send(msg)
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) CloseAndRecv() (*proto.{{.ReturnType}}, error) {
+	msg := new(proto.{{.ReturnType}})
+	resp := triple_protocol.NewResponse(msg)
+	if err := cli.ClientStreamForClient.CloseAndReceive(resp); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ClientStreamForClient.Conn()
+}
+{{end}}{{else}}
+type {{$s.ServiceName}}_{{.MethodName}}Client interface {
+	Recv() bool
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Msg() *proto.{{.ReturnType}}
+	Err() error
+	Conn() (triple_protocol.StreamingClientConn, error)
+	Close() error
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Client struct {
+	*triple_protocol.ServerStreamForClient
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Recv() bool {
+	msg := new(proto.GreetServerStreamResponse)
+	return cli.ServerStreamForClient.Receive(msg)
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Msg() *proto.{{.ReturnType}} {
+	msg := cli.ServerStreamForClient.Msg()
+	if msg == nil {
+		return new(proto.{{.ReturnType}})
+	}
+	return msg.(*proto.{{.ReturnType}})
+}
+
+func (cli *{{lower $s.ServiceName}}{{.MethodName}}Client) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ServerStreamForClient.Conn()
+}
+{{end}}{{end}}{{end}}
+
+`
+
+const MethodInfoTpl = `{{$t := .}}{{range $i, $s := .Services}}var {{.ServiceName}}_ClientInfo = client.ClientInfo{
+	InterfaceName : "{{$t.Package}}.{{.ServiceName}}",
+	MethodNames :   []string{ {{- range $j, $m := .Methods}}"{{.MethodName}}"{{if last $j (len $s.Methods)}}{{else}},{{end}}{{end -}} },
+	ClientInjectFunc : func(dubboCliRaw interface{}, cli *client.Client) {
+		dubboCli := dubboCliRaw.({{$s.ServiceName}}ClientImpl)
+		dubboCli.cli = cli
+	},
+}{{end}}
+
+`
+
+const HandlerTpl = `{{$t := .}}{{range $s := .Services}}// {{.ServiceName}}Handler is an implementation of the {{$t.ProtoPackage}}.{{.ServiceName}} service.
+type {{.ServiceName}}Handler interface {
+	{{range $s.Methods}}
+		{{.MethodName}}(context.Context, {{if .StreamsRequest}}{{$s.ServiceName}}_{{.MethodName}}Server{{else}}*proto.{{.RequestType}}{{if .StreamsReturn}},{{$s.ServiceName}}_{{.MethodName}}Server{{end}}{{end}}) {{if .StreamsReturn}}error{{else}}(*proto.{{.ReturnType}}, error){{end}}
+	{{end}}
+}
+
+func Provide{{.ServiceName}}Handler(pro *provider.Provider, hdlr {{.ServiceName}}Handler) error {
+	return pro.Provide(hdlr, &{{.ServiceName}}_ServiceInfo)
+}{{end}}
+`
+
+const ServerImplTpl = `{{$t := .}}{{range $s := .Services}}{{range .Methods}}{{if .StreamsRequest}}{{if .StreamsReturn}}
+type {{$s.ServiceName}}_{{.MethodName}}Server interface {
+	Send(*proto.{{.ReturnType}}) error
+	Recv() (*proto.{{.RequestType}}, error)
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	RequestHeader() http.Header
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Server struct {
+	*triple_protocol.BidiStream
+}
+
+func (srv *{{lower $s.ServiceName}}{{.MethodName}}Server) Send(msg *proto.{{.ReturnType}}) error {
+	return srv.BidiStream.Send(msg)
+}
+
+func (srv {{lower $s.ServiceName}}{{.MethodName}}Server) Recv() (*proto.{{.RequestType}}, error) {
+	msg := new(proto.{{.RequestType}})
+	if err := srv.BidiStream.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+{{else}}
+type {{$s.ServiceName}}_{{.MethodName}}Server interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Recv() bool
+	RequestHeader() http.Header
+	Msg() *proto.{{.RequestType}}
+	Err() error
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Server struct {
+	*triple_protocol.ClientStream
+}
+
+func (srv *{{lower $s.ServiceName}}{{.MethodName}}Server) Recv() bool {
+	msg := new(proto.GreetClientStreamRequest)
+	return srv.ClientStream.Receive(msg)
+}
+
+func (srv *{{lower $s.ServiceName}}{{.MethodName}}Server) Msg() *proto.{{.RequestType}} {
+	msgRaw := srv.ClientStream.Msg()
+	if msgRaw == nil {
+		return new(proto.{{.RequestType}})
+	}
+	return msgRaw.(*proto.{{.RequestType}})
+}
+{{end}}{{else}}
+type {{$s.ServiceName}}_{{.MethodName}}Server interface {
+	Send(*proto.GreetServerStreamResponse) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type {{lower $s.ServiceName}}{{.MethodName}}Server struct {
+	*triple_protocol.ServerStream
+}
+
+func (g *{{lower $s.ServiceName}}{{.MethodName}}Server) Send(msg *proto.{{.ReturnType}}) error {
+	return g.ServerStream.Send(msg)
+}
+{{end}}{{end}}{{end}}`
+
+const ServiceInfoTpl = `{{$t := .}}{{range $s := .Services}}var {{.ServiceName}}_ServiceInfo = provider.ServiceInfo{
+	InterfaceName: "{{$t.ProtoPackage}}.{{.ServiceName}}",
+	ServiceType:   (*{{.ServiceName}}Handler)(nil),
+	Methods: []provider.MethodInfo{ {{- range .Methods}}{{if .StreamsRequest}}{{if .StreamsReturn}}
+		{
+			Name : "{{.MethodName}}",
+			Type : constant.CallBidiStream,
+			StreamInitFunc : func(baseStream interface{}) interface{} {
+				return &{{lower $s.ServiceName}}{{.MethodName}}Server{baseStream.(*triple_protocol.BidiStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].({{$s.ServiceName}}_{{.MethodName}}Server)
+				if err := handler.({{$s.ServiceName}}Handler).{{.MethodName}}(ctx, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},{{else}}
+		{
+			Name : "{{.MethodName}}",
+			Type : constant.CallClientStream,
+			StreamInitFunc: func(baseStream interface{}) interface{} {
+				return &{{lower $s.ServiceName}}{{.MethodName}}Server{baseStream.(*triple_protocol.ClientStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].({{$s.ServiceName}}_{{.MethodName}}Server)
+				res, err := handler.({{$s.ServiceName}}Handler).{{.MethodName}}(ctx, stream)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},{{end}}{{else}}{{if .StreamsReturn}}
+		{
+			Name : "{{.MethodName}}",
+			Type : constant.CallServerStream,
+			ReqInitFunc : func() interface{} {
+				return new(proto.{{.RequestType}})
+			},
+			StreamInitFunc : func(baseStream interface{}) interface{} {
+				return &{{lower $s.ServiceName}}{{.MethodName}}Server{baseStream.(*triple_protocol.ServerStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.{{.RequestType}})
+				stream := args[1].({{$s.ServiceName}}_{{.MethodName}}Server)
+				if err := handler.({{$s.ServiceName}}Handler).{{.MethodName}}(ctx, req, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},{{else}}
+		{
+			Name : "{{.MethodName}}",
+			Type : constant.CallUnary,
+			ReqInitFunc : func() interface{} {
+				return new(proto.{{.RequestType}})
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.{{.RequestType}})
+				res, err := handler.({{$s.ServiceName}}Handler).{{.MethodName}}(ctx, req)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},{{end}}{{end}}{{end}}
+	},
+}{{end}}
+`
+
+const UnImplServiceTpl = `{{$t := .}}{{range $s := .Services}}// Unimplemented{{.ServiceName}}Handler returns CodeUnimplemented from all methods.
+type Unimplemented{{.ServiceName}}Handler struct{}
+{{range .Methods}}{{if .StreamsRequest}}{{if .StreamsReturn}}
+func (Unimplemented{{$s.ServiceName}}Handler) {{.MethodName}}(context.Context, *triple_protocol.BidiStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("{{$t.Package}}.{{$s.ServiceName}}.{{.MethodName}} is not implemented"))
+}
+{{else}}
+func (Unimplemented{{$s.ServiceName}}Handler) {{.MethodName}}(context.Context, *triple_protocol.ClientStream) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("{{$t.Package}}.{{$s.ServiceName}}.{{.MethodName}} is not implemented"))
+}
+{{end}}{{else}}{{if .StreamsReturn}}
+func (Unimplemented{{$s.ServiceName}}Handler) {{.MethodName}}(context.Context, *triple_protocol.Request, *triple_protocol.ServerStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("{{$t.Package}}.{{$s.ServiceName}}.{{.MethodName}} is not implemented"))
+}
+{{else}}
+func (Unimplemented{{$s.ServiceName}}Handler) {{.MethodName}}(context.Context, *proto.{{.RequestType}}) (*proto.{{.ReturnType}}, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("{{$t.Package}}.{{$s.ServiceName}}.{{.MethodName}} is not implemented"))
+}
+{{end}}{{end}}{{end}}{{end}}
+`
diff --git a/protocol/triple/triple-tool/gen/generator/tripleTpl_test.go b/protocol/triple/triple-tool/gen/generator/tripleTpl_test.go
new file mode 100644
index 0000000..0d366b9
--- /dev/null
+++ b/protocol/triple/triple-tool/gen/generator/tripleTpl_test.go
@@ -0,0 +1,629 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package generator
+
+import (
+	"bytes"
+	"strings"
+	"testing"
+)
+
+import (
+	"github.com/emicklei/proto"
+
+	"github.com/stretchr/testify/assert"
+)
+
+func init() {
+	protoFile := bytes.NewBufferString(testProto)
+	parser := proto.NewParser(protoFile)
+	p, _ := parser.Parse()
+
+	g := NewGenerator(Context{
+		GoModuleName: "protoc-gen-triple",
+		Src:          "../../.",
+	})
+	data, _ = g.parseProtoToTriple(p)
+}
+
+var data TripleGo
+
+//func TestPreamble(t *testing.T) {
+//	err := TplPreamble.Execute(os.Stdout, data)
+//	if err != nil {
+//		panic(err)
+//	}
+//}
+//
+//func TestPackage(t *testing.T) {
+//	err := TplPackage.Execute(os.Stdout, data)
+//	if err != nil {
+//		panic(err)
+//	}
+//}
+//
+//func TestImport(t *testing.T) {
+//	err := TplImport.Execute(os.Stdout, data)
+//	if err != nil {
+//		panic(err)
+//	}
+//}
+//
+//func TestTotalTpl(t *testing.T) {
+//	err := TplTotal.Execute(os.Stdout, data)
+//	if err != nil {
+//		panic(err)
+//	}
+//}
+//
+//func TestClientInterfaceTemplate(t *testing.T) {
+//	err := TplClientInterface.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestClientInterfaceImplTpl(t *testing.T) {
+//	err := TplClientInterfaceImpl.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestClientImplTpl(t *testing.T) {
+//	err := TplClientImpl.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestMethodInfoTpl(t *testing.T) {
+//	err := TplMethodInfo.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestHandlerTpl(t *testing.T) {
+//	err := TplHandler.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestServerImplTpl(t *testing.T) {
+//	err := TplServerImpl.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestServiceInfoImplTpl(t *testing.T) {
+//	err := TplServerInfo.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+//
+//func TestUnImplTpl(t *testing.T) {
+//	err := TplUnImpl.Execute(os.Stdout, data)
+//	if err != nil {
+//		t.Fatalf("Failed to execute template: %v", err)
+//	}
+//}
+
+func TestAll(t *testing.T) {
+	var builder strings.Builder
+	for _, tpl := range Tpls {
+		err := tpl.Execute(&builder, data)
+		if err != nil {
+			t.Fatalf("Failed to execute template: %v", err)
+		}
+	}
+	assert.Equal(t, testTripleGo, builder.String())
+}
+
+const testProto = `syntax = "proto3";
+
+package greet;
+
+option go_package = "/proto;proto";
+
+message GreetRequest {
+  string name = 1;
+}
+
+message GreetResponse {
+  string greeting = 1;
+}
+
+message GreetStreamRequest {
+  string name = 1;
+}
+
+message GreetStreamResponse {
+  string greeting = 1;
+}
+
+message GreetClientStreamRequest {
+  string name = 1;
+}
+
+message GreetClientStreamResponse {
+  string greeting = 1;
+}
+
+message GreetServerStreamRequest {
+  string name = 1;
+}
+
+message GreetServerStreamResponse {
+  string greeting = 1;
+}
+
+service GreetService {
+  rpc Greet(GreetRequest) returns (GreetResponse) {}
+  rpc GreetStream(stream GreetStreamRequest) returns (stream GreetStreamResponse) {}
+  rpc GreetClientStream(stream GreetClientStreamRequest) returns (GreetClientStreamResponse) {}
+  rpc GreetServerStream(GreetServerStreamRequest) returns (stream GreetServerStreamResponse) {}
+}`
+
+const testTripleGo = `// Code generated by protoc-gen-triple. DO NOT EDIT.
+//
+// Source: 
+package greettriple
+
+import (
+	context "context"
+	errors "errors"
+	http "net/http"
+
+	client "dubbo.apache.org/dubbo-go/v3/client"
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	proto "protoc-gen-triple/proto"
+	triple_protocol "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/provider"
+)
+
+// This is a compile-time assertion to ensure that this generated file and the connect package are
+// compatible. If you get a compiler error that this constant is not defined, this code was
+// generated with a version of connect newer than the one compiled into your binary. You can fix the
+// problem by either regenerating this code with an older version of connect or updating the connect
+// version compiled into your binary.
+const _ = triple_protocol.IsAtLeastVersion0_1_0
+
+const (
+	// GreetServiceName is the fully-qualified name of the GreetService service.
+	GreetServiceName = "greet.GreetService"
+)
+
+// These constants are the fully-qualified names of the RPCs defined in this package. They're
+// exposed at runtime as Spec.Procedure and as the final two segments of the HTTP route.
+//
+// Note that these are different from the fully-qualified method names used by
+// google.golang.org/protobuf/reflect/protoreflect. To convert from these constants to
+// reflection-formatted method names, remove the leading slash and convert the remaining slash to a
+// period.
+const (
+	// GreetServiceGreetProcedure is the fully-qualified name of the GreetService's Greet RPC.
+	GreetServiceGreetProcedure = "/greet.GreetService/Greet"
+	// GreetServiceGreetStreamProcedure is the fully-qualified name of the GreetService's GreetStream RPC.
+	GreetServiceGreetStreamProcedure = "/greet.GreetService/GreetStream"
+	// GreetServiceGreetClientStreamProcedure is the fully-qualified name of the GreetService's GreetClientStream RPC.
+	GreetServiceGreetClientStreamProcedure = "/greet.GreetService/GreetClientStream"
+	// GreetServiceGreetServerStreamProcedure is the fully-qualified name of the GreetService's GreetServerStream RPC.
+	GreetServiceGreetServerStreamProcedure = "/greet.GreetService/GreetServerStream"
+)
+
+//GreetServiceClient is a client for the greet.GreetService service.
+type GreetServiceClient interface {
+	
+		Greet(ctx context.Context, req *proto.GreetRequest, opt ...client.CallOption) (*proto.GreetResponse, error)
+	
+		GreetStream(ctx context.Context, opt ...client.CallOption) (GreetService_GreetStreamClient, error)
+	
+		GreetClientStream(ctx context.Context, opt ...client.CallOption) (GreetService_GreetClientStreamClient, error)
+	
+		GreetServerStream(ctx context.Context, req *proto.GreetServerStreamRequest, opt ...client.CallOption) (GreetService_GreetServerStreamClient, error)
+	
+}
+
+// NewGreetServiceClient constructs a client for the greettriple.GreetService service. By default, it uses
+// the Connect protocol with the binary Protobuf Codec, asks for gzipped responses, and sends
+// uncompressed requests. To use the gRPC or gRPC-Web protocols, supply the connect.WithGRPC() or
+// connect.WithGRPCWeb() options.
+//
+// The URL supplied here should be the base URL for the Connect or gRPC server (for example,
+// http://api.acme.com or https://acme.com/grpc).
+func NewGreetServiceClient(cli *client.Client) (GreetServiceClient, error) {
+	if err := cli.Init(&GreetService_ClientInfo); err != nil {
+		return nil, err
+	}
+	return &GreetServiceClientImpl{
+		cli: cli,
+	}, nil
+}
+
+func SetConsumerService(srv common.RPCService) {
+	config.SetClientInfoService(&GreetService_ClientInfo, srv)
+}
+
+// GreetServiceClientImpl implements GreetServiceClient.
+type GreetServiceClientImpl struct {
+	cli *client.Client
+}
+
+func (c *GreetServiceClientImpl) Greet(ctx context.Context, req *proto.GreetRequest, opts ...client.CallOption) (*proto.GreetResponse, error) {
+	triReq := triple_protocol.NewRequest(req)
+	resp := new(proto.GreetResponse)
+	triResp := triple_protocol.NewResponse(resp)
+	if err := c.cli.CallUnary(ctx, triReq, triResp, "greet.GreetService", "Greet", opts...); err != nil {
+		return nil, err
+	}
+	return resp, nil
+}
+
+func (c *GreetServiceClientImpl) GreetStream(ctx context.Context, opts ...client.CallOption)(GreetService_GreetStreamClient,error) {
+	stream, err := c.cli.CallBidiStream(ctx, "greet.GreetService", "GreetStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.BidiStreamForClient)
+	return &greetServiceGreetStreamClient{rawStream}, nil
+}
+
+func (c *GreetServiceClientImpl) GreetClientStream(ctx context.Context, opts ...client.CallOption) (GreetService_GreetClientStreamClient, error) {
+	stream, err := c.cli.CallClientStream(ctx, "greet.GreetService", "GreetClientStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ClientStreamForClient)
+	return &greetServiceGreetClientStreamClient{rawStream}, nil
+}
+
+func (c *GreetServiceClientImpl) GreetServerStream(ctx context.Context, req *proto.GreetServerStreamRequest, opts ...client.CallOption) (GreetService_GreetServerStreamClient, error) {
+	triReq := triple_protocol.NewRequest(req)
+	stream, err := c.cli.CallServerStream(ctx, triReq, "greet.GreetService", "GreetServerStream", opts...)
+	if err != nil {
+		return nil, err
+	}
+	rawStream := stream.(*triple_protocol.ServerStreamForClient)
+	return &greetServiceGreetServerStreamClient{rawStream}, nil
+}
+
+
+type GreetService_GreetClient interface {
+	Recv() bool
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Msg() *proto.GreetResponse
+	Err() error
+	Conn() (triple_protocol.StreamingClientConn, error)
+	Close() error
+}
+
+type greetServiceGreetClient struct {
+	*triple_protocol.ServerStreamForClient
+}
+
+func (cli *greetServiceGreetClient) Recv() bool {
+	msg := new(proto.GreetServerStreamResponse)
+	return cli.ServerStreamForClient.Receive(msg)
+}
+
+func (cli *greetServiceGreetClient) Msg() *proto.GreetResponse {
+	msg := cli.ServerStreamForClient.Msg()
+	if msg == nil {
+		return new(proto.GreetResponse)
+	}
+	return msg.(*proto.GreetResponse)
+}
+
+func (cli *greetServiceGreetClient) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ServerStreamForClient.Conn()
+}
+ 
+type GreetService_GreetStreamClient interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.GreetStreamRequest) error
+	RequestHeader() http.Header
+	CloseRequest() error
+	Recv() (*proto.GreetStreamResponse, error)
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	CloseResponse() error
+}
+
+type greetServiceGreetStreamClient struct {
+	*triple_protocol.BidiStreamForClient
+}
+
+func (cli *greetServiceGreetStreamClient) Send(msg *proto.GreetStreamRequest) error {
+	return cli.BidiStreamForClient.Send(msg)
+}
+
+func (cli *greetServiceGreetStreamClient) Recv() (*proto.GreetStreamResponse, error) {
+	msg := new(proto.GreetStreamResponse)
+	if err := cli.BidiStreamForClient.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+type GreetService_GreetClientStreamClient interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Send(*proto.GreetClientStreamRequest) error
+	RequestHeader() http.Header
+	CloseAndRecv() (*proto.GreetClientStreamResponse, error)
+	Conn() (triple_protocol.StreamingClientConn, error)
+}
+
+type greetServiceGreetClientStreamClient struct {
+	*triple_protocol.ClientStreamForClient
+}
+
+func (cli *greetServiceGreetClientStreamClient) Send(msg *proto.GreetClientStreamRequest) error {
+	return cli.ClientStreamForClient.Send(msg)
+}
+
+func (cli *greetServiceGreetClientStreamClient) CloseAndRecv() (*proto.GreetClientStreamResponse, error) {
+	msg := new(proto.GreetClientStreamResponse)
+	resp := triple_protocol.NewResponse(msg)
+	if err := cli.ClientStreamForClient.CloseAndReceive(resp); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+func (cli *greetServiceGreetClientStreamClient) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ClientStreamForClient.Conn()
+}
+
+type GreetService_GreetServerStreamClient interface {
+	Recv() bool
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Msg() *proto.GreetServerStreamResponse
+	Err() error
+	Conn() (triple_protocol.StreamingClientConn, error)
+	Close() error
+}
+
+type greetServiceGreetServerStreamClient struct {
+	*triple_protocol.ServerStreamForClient
+}
+
+func (cli *greetServiceGreetServerStreamClient) Recv() bool {
+	msg := new(proto.GreetServerStreamResponse)
+	return cli.ServerStreamForClient.Receive(msg)
+}
+
+func (cli *greetServiceGreetServerStreamClient) Msg() *proto.GreetServerStreamResponse {
+	msg := cli.ServerStreamForClient.Msg()
+	if msg == nil {
+		return new(proto.GreetServerStreamResponse)
+	}
+	return msg.(*proto.GreetServerStreamResponse)
+}
+
+func (cli *greetServiceGreetServerStreamClient) Conn() (triple_protocol.StreamingClientConn, error) {
+	return cli.ServerStreamForClient.Conn()
+}
+
+
+var GreetService_ClientInfo = client.ClientInfo{
+	InterfaceName : "greettriple.GreetService",
+	MethodNames :   []string{"Greet","GreetStream","GreetClientStream","GreetServerStream"},
+	ClientInjectFunc : func(dubboCliRaw interface{}, cli *client.Client) {
+		dubboCli := dubboCliRaw.(GreetServiceClientImpl)
+		dubboCli.cli = cli
+	},
+}
+
+// GreetServiceHandler is an implementation of the greet.GreetService service.
+type GreetServiceHandler interface {
+	
+		Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error)
+	
+		GreetStream(context.Context, GreetService_GreetStreamServer) error
+	
+		GreetClientStream(context.Context, GreetService_GreetClientStreamServer) (*proto.GreetClientStreamResponse, error)
+	
+		GreetServerStream(context.Context, *proto.GreetServerStreamRequest,GreetService_GreetServerStreamServer) error
+	
+}
+
+func ProvideGreetServiceHandler(pro *provider.Provider, hdlr GreetServiceHandler) error {
+	return pro.Provide(hdlr, &GreetService_ServiceInfo)
+}
+
+type GreetService_GreetServer interface {
+	Send(*proto.GreetServerStreamResponse) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetServer struct {
+	*triple_protocol.ServerStream
+}
+
+func (g *greetServiceGreetServer) Send(msg *proto.GreetResponse) error {
+	return g.ServerStream.Send(msg)
+}
+
+type GreetService_GreetStreamServer interface {
+	Send(*proto.GreetStreamResponse) error
+	Recv() (*proto.GreetStreamRequest, error)
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	RequestHeader() http.Header
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetStreamServer struct {
+	*triple_protocol.BidiStream
+}
+
+func (srv *greetServiceGreetStreamServer) Send(msg *proto.GreetStreamResponse) error {
+	return srv.BidiStream.Send(msg)
+}
+
+func (srv greetServiceGreetStreamServer) Recv() (*proto.GreetStreamRequest, error) {
+	msg := new(proto.GreetStreamRequest)
+	if err := srv.BidiStream.Receive(msg); err != nil {
+		return nil, err
+	}
+	return msg, nil
+}
+
+type GreetService_GreetClientStreamServer interface {
+	Spec() triple_protocol.Spec
+	Peer() triple_protocol.Peer
+	Recv() bool
+	RequestHeader() http.Header
+	Msg() *proto.GreetClientStreamRequest
+	Err() error
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetClientStreamServer struct {
+	*triple_protocol.ClientStream
+}
+
+func (srv *greetServiceGreetClientStreamServer) Recv() bool {
+	msg := new(proto.GreetClientStreamRequest)
+	return srv.ClientStream.Receive(msg)
+}
+
+func (srv *greetServiceGreetClientStreamServer) Msg() *proto.GreetClientStreamRequest {
+	msgRaw := srv.ClientStream.Msg()
+	if msgRaw == nil {
+		return new(proto.GreetClientStreamRequest)
+	}
+	return msgRaw.(*proto.GreetClientStreamRequest)
+}
+
+type GreetService_GreetServerStreamServer interface {
+	Send(*proto.GreetServerStreamResponse) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	Conn() triple_protocol.StreamingHandlerConn
+}
+
+type greetServiceGreetServerStreamServer struct {
+	*triple_protocol.ServerStream
+}
+
+func (g *greetServiceGreetServerStreamServer) Send(msg *proto.GreetServerStreamResponse) error {
+	return g.ServerStream.Send(msg)
+}
+var GreetService_ServiceInfo = provider.ServiceInfo{
+	InterfaceName: "greet.GreetService",
+	ServiceType:   (*GreetServiceHandler)(nil),
+	Methods: []provider.MethodInfo{
+		{
+			Name : "Greet",
+			Type : constant.CallUnary,
+			ReqInitFunc : func() interface{} {
+				return new(proto.GreetRequest)
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.GreetRequest)
+				res, err := handler.(GreetServiceHandler).Greet(ctx, req)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},
+		{
+			Name : "GreetStream",
+			Type : constant.CallBidiStream,
+			StreamInitFunc : func(baseStream interface{}) interface{} {
+				return &greetServiceGreetStreamServer{baseStream.(*triple_protocol.BidiStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].(GreetService_GreetStreamServer)
+				if err := handler.(GreetServiceHandler).GreetStream(ctx, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},
+		{
+			Name : "GreetClientStream",
+			Type : constant.CallClientStream,
+			StreamInitFunc: func(baseStream interface{}) interface{} {
+				return &greetServiceGreetClientStreamServer{baseStream.(*triple_protocol.ClientStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				stream := args[0].(GreetService_GreetClientStreamServer)
+				res, err := handler.(GreetServiceHandler).GreetClientStream(ctx, stream)
+				if err != nil {
+					return nil, err
+				}
+				return triple_protocol.NewResponse(res), nil
+			},
+		},
+		{
+			Name : "GreetServerStream",
+			Type : constant.CallServerStream,
+			ReqInitFunc : func() interface{} {
+				return new(proto.GreetServerStreamRequest)
+			},
+			StreamInitFunc : func(baseStream interface{}) interface{} {
+				return &greetServiceGreetServerStreamServer{baseStream.(*triple_protocol.ServerStream)}
+			},
+			MethodFunc : func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error) {
+				req := args[0].(*proto.GreetServerStreamRequest)
+				stream := args[1].(GreetService_GreetServerStreamServer)
+				if err := handler.(GreetServiceHandler).GreetServerStream(ctx, req, stream); err != nil {
+					return nil, err
+				}
+				return nil, nil
+			},
+		},
+	},
+}
+// UnimplementedGreetServiceHandler returns CodeUnimplemented from all methods.
+type UnimplementedGreetServiceHandler struct{}
+
+func (UnimplementedGreetServiceHandler) Greet(context.Context, *proto.GreetRequest) (*proto.GreetResponse, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greettriple.GreetService.Greet is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetStream(context.Context, *triple_protocol.BidiStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greettriple.GreetService.GreetStream is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetClientStream(context.Context, *triple_protocol.ClientStream) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greettriple.GreetService.GreetClientStream is not implemented"))
+}
+
+func (UnimplementedGreetServiceHandler) GreetServerStream(context.Context, *triple_protocol.Request, *triple_protocol.ServerStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("greettriple.GreetService.GreetServerStream is not implemented"))
+}
+
+`
diff --git a/protocol/triple/triple-tool/go.mod b/protocol/triple/triple-tool/go.mod
new file mode 100644
index 0000000..676a82e
--- /dev/null
+++ b/protocol/triple/triple-tool/go.mod
@@ -0,0 +1,19 @@
+module dubbo.apache.org/dubbo-go/v3/triple-tool
+
+go 1.20
+
+require (
+	github.com/emicklei/proto v1.12.1
+	github.com/golang/protobuf v1.5.0
+	github.com/spf13/cobra v1.7.0
+	github.com/stretchr/testify v1.8.4
+	google.golang.org/protobuf v1.31.0
+)
+
+require (
+	github.com/davecgh/go-spew v1.1.1 // indirect
+	github.com/inconshreveable/mousetrap v1.1.0 // indirect
+	github.com/pmezard/go-difflib v1.0.0 // indirect
+	github.com/spf13/pflag v1.0.5 // indirect
+	gopkg.in/yaml.v3 v3.0.1 // indirect
+)
diff --git a/protocol/triple/triple-tool/go.sum b/protocol/triple/triple-tool/go.sum
new file mode 100644
index 0000000..a4a8389
--- /dev/null
+++ b/protocol/triple/triple-tool/go.sum
@@ -0,0 +1,29 @@
+github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o=
+github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c=
+github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38=
+github.com/emicklei/proto v1.12.1 h1:6n/Z2pZAnBwuhU66Gs8160B8rrrYKo7h2F2sCOnNceE=
+github.com/emicklei/proto v1.12.1/go.mod h1:rn1FgRS/FANiZdD2djyH7TMA9jdRDcYQ9IEN9yvjX0A=
+github.com/golang/protobuf v1.5.0 h1:LUVKkCeviFUMKqHa4tXIIij/lbhnMbP7Fn5wKdKkRh4=
+github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk=
+github.com/google/go-cmp v0.5.5 h1:Khx7svrCpmxxtHBq5j2mp/xVjsi8hQMfNLvJFAlrGgU=
+github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE=
+github.com/inconshreveable/mousetrap v1.1.0 h1:wN+x4NVGpMsO7ErUn/mUI3vEoE6Jt13X2s0bqwp9tc8=
+github.com/inconshreveable/mousetrap v1.1.0/go.mod h1:vpF70FUmC8bwa3OWnCshd2FqLfsEA9PFc4w1p2J65bw=
+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/russross/blackfriday/v2 v2.1.0/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM=
+github.com/spf13/cobra v1.7.0 h1:hyqWnYt1ZQShIddO5kBpj3vu05/++x6tJ6dg8EC572I=
+github.com/spf13/cobra v1.7.0/go.mod h1:uLxZILRyS/50WlhOIKD7W6V5bgeIt+4sICxh6uRMrb0=
+github.com/spf13/pflag v1.0.5 h1:iy+VFUOCP1a+8yFto/drg2CJ5u0yRoB7fZw3DKv/JXA=
+github.com/spf13/pflag v1.0.5/go.mod h1:McXfInJRrz4CZXVZOBLb0bTZqETkiAhM9Iw0y3An2Bg=
+github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk=
+github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4=
+golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0=
+google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw=
+google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8=
+google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405 h1:yhCVgyC4o1eVCa2tZl7eS0r+SDo693bJlVdllGtEeKM=
+gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0=
+gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA=
+gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM=
diff --git a/protocol/triple/triple-tool/internal/proto/greet.proto b/protocol/triple/triple-tool/internal/proto/greet.proto
new file mode 100644
index 0000000..f4b49b4
--- /dev/null
+++ b/protocol/triple/triple-tool/internal/proto/greet.proto
@@ -0,0 +1,44 @@
+syntax = "proto3";
+
+package greet;
+
+option go_package = "/internal/proto;greet";
+
+message GreetRequest {
+  string name = 1;
+}
+
+message GreetResponse {
+  string greeting = 1;
+}
+
+message GreetStreamRequest {
+  string name = 1;
+}
+
+message GreetStreamResponse {
+  string greeting = 1;
+}
+
+message GreetClientStreamRequest {
+  string name = 1;
+}
+
+message GreetClientStreamResponse {
+  string greeting = 1;
+}
+
+message GreetServerStreamRequest {
+  string name = 1;
+}
+
+message GreetServerStreamResponse {
+  string greeting = 1;
+}
+
+service GreetService {
+  rpc Greet(GreetRequest) returns (GreetResponse) {}
+  rpc GreetStream(stream GreetStreamRequest) returns (stream GreetStreamResponse) {}
+  rpc GreetClientStream(stream GreetClientStreamRequest) returns (GreetClientStreamResponse) {}
+  rpc GreetServerStream(GreetServerStreamRequest) returns (stream GreetServerStreamResponse) {}
+}
\ No newline at end of file
diff --git a/protocol/triple/triple-tool/internal/version/version.go b/protocol/triple/triple-tool/internal/version/version.go
new file mode 100644
index 0000000..35bc9e3
--- /dev/null
+++ b/protocol/triple/triple-tool/internal/version/version.go
@@ -0,0 +1,20 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package version
+
+const Version = "1.7.0-dev"
diff --git a/protocol/triple/triple-tool/main.go b/protocol/triple/triple-tool/main.go
new file mode 100644
index 0000000..1efa5b9
--- /dev/null
+++ b/protocol/triple/triple-tool/main.go
@@ -0,0 +1,26 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/cmd"
+)
+
+func main() {
+	cmd.Execute()
+}
diff --git a/protocol/triple/triple-tool/protoc-gen-triple/main.go b/protocol/triple/triple-tool/protoc-gen-triple/main.go
new file mode 100644
index 0000000..155bee7
--- /dev/null
+++ b/protocol/triple/triple-tool/protoc-gen-triple/main.go
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package main
+
+import (
+	"fmt"
+	"os"
+)
+
+import (
+	"google.golang.org/protobuf/compiler/protogen"
+
+	"google.golang.org/protobuf/types/pluginpb"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/gen/generator"
+	"dubbo.apache.org/dubbo-go/v3/triple-tool/internal/version"
+)
+
+const (
+	usage = "See https://connect.build/docs/go/getting-started to learn how to use this plugin.\n\nFlags:\n  -h, --help\tPrint this help and exit.\n      --version\tPrint the version and exit."
+)
+
+func main() {
+	if len(os.Args) == 2 && os.Args[1] == "--version" {
+		fmt.Fprintln(os.Stdout, version.Version)
+		os.Exit(0)
+	}
+	if len(os.Args) == 2 && (os.Args[1] == "-h" || os.Args[1] == "--help") {
+		fmt.Fprintln(os.Stdout, usage)
+		os.Exit(0)
+	}
+	if len(os.Args) != 1 {
+		fmt.Fprintln(os.Stderr, usage)
+		os.Exit(1)
+	}
+
+	protogen.Options{}.Run(
+		func(plugin *protogen.Plugin) error {
+			plugin.SupportedFeatures = uint64(pluginpb.CodeGeneratorResponse_FEATURE_PROTO3_OPTIONAL)
+			for _, file := range plugin.Files {
+				if file.Generate {
+					tripleGo, err := generator.ProcessProtoFile(file.Proto)
+					if err != nil {
+						return err
+					}
+					return generator.GenTripleFile(tripleGo)
+				}
+			}
+			return nil
+		},
+	)
+}
diff --git a/protocol/triple/triple-tool/util/exec.go b/protocol/triple/triple-tool/util/exec.go
new file mode 100644
index 0000000..825fc58
--- /dev/null
+++ b/protocol/triple/triple-tool/util/exec.go
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package util
+
+import (
+	"fmt"
+	"os/exec"
+	"runtime"
+)
+
+func Exec(arg, dir string) (string, error) {
+	osEnv := runtime.GOOS
+	var cmd *exec.Cmd
+	switch osEnv {
+	case "darwin", "linux":
+		cmd = exec.Command("sh", "-c", arg)
+	case "windows":
+		cmd = exec.Command("cmd.exe", "/c", arg)
+	default:
+		return "", fmt.Errorf("unexpected os: %v", osEnv)
+	}
+	if len(dir) > 0 {
+		cmd.Dir = dir
+	}
+	output, err := cmd.CombinedOutput()
+	return string(output), err
+}
diff --git a/protocol/triple/triple-tool/util/module.go b/protocol/triple/triple-tool/util/module.go
new file mode 100644
index 0000000..a399d83
--- /dev/null
+++ b/protocol/triple/triple-tool/util/module.go
@@ -0,0 +1,32 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package util
+
+import (
+	"strings"
+)
+
+func GetModuleName() (string, error) {
+	output, err := Exec("go list -m", "./")
+	if err != nil {
+		return "", err
+	}
+
+	moduleName := strings.TrimSpace(output)
+	return moduleName, nil
+}
diff --git a/protocol/triple/triple.go b/protocol/triple/triple.go
new file mode 100644
index 0000000..cb64aa4
--- /dev/null
+++ b/protocol/triple/triple.go
@@ -0,0 +1,140 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"sync"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+const (
+	// TRIPLE protocol name
+	TRIPLE = "tri"
+)
+
+var (
+	tripleProtocol *TripleProtocol
+)
+
+func init() {
+	extension.SetProtocol(TRIPLE, GetProtocol)
+}
+
+type TripleProtocol struct {
+	protocol.BaseProtocol
+	serverLock sync.Mutex
+	serverMap  map[string]*Server
+}
+
+// Export TRIPLE service for remote invocation
+func (tp *TripleProtocol) Export(invoker protocol.Invoker) protocol.Exporter {
+	url := invoker.GetURL()
+	serviceKey := url.ServiceKey()
+	// todo: retrieve this info from url
+	var info *server.ServiceInfo
+	infoRaw, ok := url.Attributes[constant.ServiceInfoKey]
+	if ok {
+		info = infoRaw.(*server.ServiceInfo)
+	}
+	exporter := NewTripleExporter(serviceKey, invoker, tp.ExporterMap())
+	tp.SetExporterMap(serviceKey, exporter)
+	logger.Infof("[TRIPLE Protocol] Export service: %s", url.String())
+	tp.openServer(invoker, info)
+	return exporter
+}
+
+// *Important*. This function is only for testing. When server package is finished, remove this function
+// and modify related tests.
+func (tp *TripleProtocol) exportForTest(invoker protocol.Invoker, info *server.ServiceInfo) protocol.Exporter {
+	url := invoker.GetURL()
+	serviceKey := url.ServiceKey()
+	// todo: retrieve this info from url
+	exporter := NewTripleExporter(serviceKey, invoker, tp.ExporterMap())
+	tp.SetExporterMap(serviceKey, exporter)
+	logger.Infof("[TRIPLE Protocol] Export service: %s", url.String())
+	tp.openServer(invoker, info)
+	return exporter
+}
+
+func (tp *TripleProtocol) openServer(invoker protocol.Invoker, info *server.ServiceInfo) {
+	url := invoker.GetURL()
+	tp.serverLock.Lock()
+	defer tp.serverLock.Unlock()
+
+	if _, ok := tp.serverMap[url.Location]; ok {
+		return
+	}
+
+	if _, ok := tp.ExporterMap().Load(url.ServiceKey()); !ok {
+		panic("[TRIPLE Protocol]" + url.Key() + "is not existing")
+	}
+
+	srv := NewServer()
+	tp.serverMap[url.Location] = srv
+	srv.Start(invoker, info)
+}
+
+// Refer a remote triple service
+func (tp *TripleProtocol) Refer(url *common.URL) protocol.Invoker {
+	invoker, err := NewTripleInvoker(url)
+	if err != nil {
+		logger.Warnf("can't dial the server: %s", url.Key())
+		return nil
+	}
+	tp.SetInvokers(invoker)
+	logger.Infof("[TRIPLE Protocol] Refer service: %s", url.String())
+	return invoker
+}
+
+func (tp *TripleProtocol) Destroy() {
+	logger.Infof("TripleProtocol destroy.")
+
+	tp.serverLock.Lock()
+	defer tp.serverLock.Unlock()
+	for key, server := range tp.serverMap {
+		delete(tp.serverMap, key)
+		server.GracefulStop()
+	}
+
+	tp.BaseProtocol.Destroy()
+}
+
+func NewTripleProtocol() *TripleProtocol {
+	return &TripleProtocol{
+		BaseProtocol: protocol.NewBaseProtocol(),
+		serverMap:    make(map[string]*Server),
+	}
+}
+
+func GetProtocol() protocol.Protocol {
+	if tripleProtocol == nil {
+		tripleProtocol = NewTripleProtocol()
+	}
+	return tripleProtocol
+}
diff --git a/protocol/triple/triple_exporter.go b/protocol/triple/triple_exporter.go
new file mode 100644
index 0000000..2dd45ad
--- /dev/null
+++ b/protocol/triple/triple_exporter.go
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"sync"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+)
+
+// TripleExporter wraps BaseExporter
+type TripleExporter struct {
+	*protocol.BaseExporter
+}
+
+func NewTripleExporter(key string, invoker protocol.Invoker, exporterMap *sync.Map) *TripleExporter {
+	return &TripleExporter{
+		BaseExporter: protocol.NewBaseExporter(key, invoker, exporterMap),
+	}
+}
+
+// UnExport and unregister Triple service from registry and memory.
+func (te *TripleExporter) UnExport() {
+	interfaceName := te.GetInvoker().GetURL().GetParam(constant.InterfaceKey, "")
+	te.BaseExporter.UnExport()
+	// todo: move UnRegister logic to a better place
+	err := common.ServiceMap.UnRegister(interfaceName, TRIPLE, te.GetInvoker().GetURL().ServiceKey())
+	if err != nil {
+		logger.Errorf("[GrpcNewExporter.UnExport] error: %v", err)
+	}
+}
diff --git a/protocol/triple/triple_invoker.go b/protocol/triple/triple_invoker.go
new file mode 100644
index 0000000..9bcc4ab
--- /dev/null
+++ b/protocol/triple/triple_invoker.go
@@ -0,0 +1,155 @@
+package triple
+
+import (
+	"context"
+	"fmt"
+	"sync"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+)
+
+type TripleInvoker struct {
+	protocol.BaseInvoker
+	quitOnce      sync.Once
+	clientGuard   *sync.RWMutex
+	clientManager *clientManager
+}
+
+func (gni *TripleInvoker) setClientManager(cm *clientManager) {
+	gni.clientGuard.Lock()
+	defer gni.clientGuard.Unlock()
+
+	gni.clientManager = cm
+}
+
+func (gni *TripleInvoker) getClientManager() *clientManager {
+	gni.clientGuard.RLock()
+	defer gni.clientGuard.RUnlock()
+
+	return gni.clientManager
+}
+
+// Invoke is used to call client-side method.
+func (ti *TripleInvoker) Invoke(ctx context.Context, invocation protocol.Invocation) protocol.Result {
+	var result protocol.RPCResult
+
+	if !ti.BaseInvoker.IsAvailable() {
+		// Generally, the case will not happen, because the invoker has been removed
+		// from the invoker list before destroy,so no new request will enter the destroyed invoker
+		logger.Warnf("TripleInvoker is destroyed")
+		result.SetError(protocol.ErrDestroyedInvoker)
+		return &result
+	}
+
+	ti.clientGuard.RLock()
+	defer ti.clientGuard.RUnlock()
+
+	if ti.clientManager == nil {
+		result.SetError(protocol.ErrClientClosed)
+		return &result
+	}
+	callTypeRaw, ok := invocation.GetAttribute(constant.CallTypeKey)
+	if !ok {
+		panic("Miss CallType to invoke TripleInvoker")
+	}
+	callType, ok := callTypeRaw.(string)
+	if !ok {
+		panic(fmt.Sprintf("CallType should be string, but got %v", callTypeRaw))
+	}
+	// please refer to methods of client.Client or code generated by new triple for the usage of inRaw and inRawLen
+	// e.g. Client.CallUnary(... req, resp []interface, ...)
+	// inRaw represents req and resp, inRawLen represents 2.
+	inRaw := invocation.ParameterRawValues()
+	inRawLen := len(inRaw)
+	method := invocation.MethodName()
+	// todo(DMwangnima): process headers(metadata) passed in
+	switch callType {
+	case constant.CallUnary:
+		if len(inRaw) != 2 {
+			panic(fmt.Sprintf("Wrong parameter Values number for CallUnary, want 2, but got %d", inRawLen))
+		}
+		if err := ti.clientManager.callUnary(ctx, method, inRaw[0], inRaw[1]); err != nil {
+			result.SetError(err)
+			return &result
+		}
+	case constant.CallClientStream:
+		stream, err := ti.clientManager.callClientStream(ctx, method)
+		if err != nil {
+			result.SetError(err)
+			return &result
+		}
+		result.SetResult(stream)
+	case constant.CallServerStream:
+		if inRawLen != 1 {
+			panic(fmt.Sprintf("Wrong parameter Values number for CallServerStream, want 1, but got %d", inRawLen))
+		}
+		stream, err := ti.clientManager.callServerStream(ctx, method, inRaw[0])
+		if err != nil {
+			result.Err = err
+			return &result
+		}
+		result.SetResult(stream)
+	case constant.CallBidiStream:
+		stream, err := ti.clientManager.callBidiStream(ctx, method)
+		if err != nil {
+			result.Err = err
+			return &result
+		}
+		result.SetResult(stream)
+	default:
+		panic(fmt.Sprintf("Unsupported CallType: %s", callType))
+	}
+
+	return &result
+}
+
+// IsAvailable get available status
+func (gni *TripleInvoker) IsAvailable() bool {
+	if gni.getClientManager() != nil {
+		return gni.BaseInvoker.IsAvailable()
+	}
+
+	return false
+}
+
+// IsDestroyed get destroyed status
+func (gni *TripleInvoker) IsDestroyed() bool {
+	if gni.getClientManager() != nil {
+		return gni.BaseInvoker.IsDestroyed()
+	}
+
+	return false
+}
+
+// Destroy will destroy Triple's invoker and client, so it is only called once
+func (ti *TripleInvoker) Destroy() {
+	ti.quitOnce.Do(func() {
+		ti.BaseInvoker.Destroy()
+		if cm := ti.getClientManager(); cm != nil {
+			ti.setClientManager(nil)
+			// todo:// find a better way to destroy these resources
+			cm.close()
+		}
+	})
+}
+
+func NewTripleInvoker(url *common.URL) (*TripleInvoker, error) {
+	cm, err := newClientManager(url)
+	if err != nil {
+		return nil, err
+	}
+	return &TripleInvoker{
+		BaseInvoker:   *protocol.NewBaseInvoker(url),
+		quitOnce:      sync.Once{},
+		clientGuard:   &sync.RWMutex{},
+		clientManager: cm,
+	}, nil
+}
diff --git a/protocol/triple/triple_protocol/Makefile b/protocol/triple/triple_protocol/Makefile
new file mode 100644
index 0000000..a872b47
--- /dev/null
+++ b/protocol/triple/triple_protocol/Makefile
@@ -0,0 +1,94 @@
+# See https://tech.davis-hansson.com/p/make/
+SHELL := bash
+.DELETE_ON_ERROR:
+.SHELLFLAGS := -eu -o pipefail -c
+.DEFAULT_GOAL := all
+MAKEFLAGS += --warn-undefined-variables
+MAKEFLAGS += --no-builtin-rules
+MAKEFLAGS += --no-print-directory
+BIN := .tmp/bin
+COPYRIGHT_YEARS := 2021-2023
+LICENSE_IGNORE := --ignore /testdata/
+# Set to use a different compiler. For example, `GO=go1.18rc1 make test`.
+GO ?= go
+
+.PHONY: help
+help: ## Describe useful make targets
+	@grep -E '^[a-zA-Z_-]+:.*?## .*$$' $(MAKEFILE_LIST) | sort | awk 'BEGIN {FS = ":.*?## "}; {printf "%-30s %s\n", $$1, $$2}'
+
+.PHONY: all
+all: ## Build, test, and lint (default)
+	$(MAKE) test
+	$(MAKE) lint
+
+.PHONY: clean
+clean: ## Delete intermediate build artifacts
+	@# -X only removes untracked files, -d recurses into directories, -f actually removes files/dirs
+	git clean -Xdf
+
+.PHONY: test
+test: build ## Run unit tests
+	$(GO) test -vet=off -race -cover ./...
+
+.PHONY: build
+build: generate ## Build all packages
+	$(GO) build ./...
+
+.PHONY: install
+install: ## Install all binaries
+	$(GO) install ./...
+
+.PHONY: lint
+lint: $(BIN)/golangci-lint $(BIN)/buf ## Lint Go and protobuf
+	test -z "$$($(BIN)/buf format -d . | tee /dev/stderr)"
+	$(GO) vet ./...
+	$(BIN)/golangci-lint run
+	$(BIN)/buf lint
+	$(BIN)/buf format -d --exit-code
+
+.PHONY: lintfix
+lintfix: $(BIN)/golangci-lint $(BIN)/buf ## Automatically fix some lint errors
+	$(BIN)/golangci-lint run --fix
+	$(BIN)/buf format -w
+
+.PHONY: generate
+generate: $(BIN)/buf $(BIN)/protoc-gen-go $(BIN)/protoc-gen-connect-go $(BIN)/license-header ## Regenerate code and licenses
+	rm -rf internal/gen
+	PATH=$(abspath $(BIN)) $(BIN)/buf generate
+	$(BIN)/license-header \
+		--license-type apache \
+		--copyright-holder "Buf Technologies, Inc." \
+		--year-range "$(COPYRIGHT_YEARS)" $(LICENSE_IGNORE)
+
+.PHONY: upgrade
+upgrade: ## Upgrade dependencies
+	go get -u -t ./... && go mod tidy -v
+
+.PHONY: checkgenerate
+checkgenerate:
+	@# Used in CI to verify that `make generate` doesn't produce a diff.
+	test -z "$$(git status --porcelain | tee /dev/stderr)"
+
+.PHONY: $(BIN)/protoc-gen-connect-go
+$(BIN)/protoc-gen-connect-go:
+	@mkdir -p $(@D)
+	$(GO) build -o $(@) ./cmd/protoc-gen-connect-go
+
+$(BIN)/buf: Makefile
+	@mkdir -p $(@D)
+	GOBIN=$(abspath $(@D)) $(GO) install github.com/bufbuild/buf/cmd/buf@v1.13.1
+
+$(BIN)/license-header: Makefile
+	@mkdir -p $(@D)
+	GOBIN=$(abspath $(@D)) $(GO) install \
+		  github.com/bufbuild/buf/private/pkg/licenseheader/cmd/license-header@90fa81df0e9ef86ed505956be1ab1e8ccd49aa52
+
+$(BIN)/golangci-lint: Makefile
+	@mkdir -p $(@D)
+	GOBIN=$(abspath $(@D)) $(GO) install github.com/golangci/golangci-lint/cmd/golangci-lint@v1.51.0
+
+$(BIN)/protoc-gen-go: Makefile go.mod
+	@mkdir -p $(@D)
+	@# The version of protoc-gen-go is determined by the version in go.mod
+	GOBIN=$(abspath $(@D)) $(GO) install google.golang.org/protobuf/cmd/protoc-gen-go
+
diff --git a/protocol/triple/triple_protocol/bench_test.go b/protocol/triple/triple_protocol/bench_test.go
new file mode 100644
index 0000000..3ceed60
--- /dev/null
+++ b/protocol/triple/triple_protocol/bench_test.go
@@ -0,0 +1,178 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"bytes"
+	"compress/gzip"
+	"context"
+	"encoding/json"
+	"io"
+	"net/http"
+	"net/http/httptest"
+	"strings"
+	"testing"
+)
+
+import (
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func BenchmarkTriple(b *testing.B) {
+	mux := http.NewServeMux()
+	mux.Handle(
+		pingv1connect.NewPingServiceHandler(
+			&ExamplePingServer{},
+		),
+	)
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	defer server.Close()
+
+	httpClient := server.Client()
+	httpTransport, ok := httpClient.Transport.(*http.Transport)
+	assert.True(b, ok)
+	httpTransport.DisableCompression = true
+
+	client := pingv1connect.NewPingServiceClient(
+		httpClient,
+		server.URL,
+		tri.WithSendGzip(),
+	)
+	twoMiB := strings.Repeat("a", 2*1024*1024)
+	b.ResetTimer()
+
+	b.Run("unary", func(b *testing.B) {
+		b.RunParallel(func(pb *testing.PB) {
+			for pb.Next() {
+				_ = client.Ping(
+					context.Background(),
+					tri.NewRequest(&pingv1.PingRequest{Text: twoMiB}),
+					tri.NewResponse(&pingv1.PingResponse{}),
+				)
+			}
+		})
+	})
+}
+
+type ping struct {
+	Text string `json:"text"`
+}
+
+func BenchmarkREST(b *testing.B) {
+	handler := func(writer http.ResponseWriter, request *http.Request) {
+		defer request.Body.Close()
+		defer func() {
+			_, err := io.Copy(io.Discard, request.Body)
+			assert.Nil(b, err)
+		}()
+		writer.Header().Set("Content-Type", "application/json")
+		var body io.Reader = request.Body
+		if request.Header.Get("Content-Encoding") == "gzip" {
+			gzipReader, err := gzip.NewReader(body)
+			if err != nil {
+				b.Fatalf("get gzip reader: %v", err)
+			}
+			defer gzipReader.Close()
+			body = gzipReader
+		}
+		var out io.Writer = writer
+		if strings.Contains(request.Header.Get("Accept-Encoding"), "gzip") {
+			writer.Header().Set("Content-Encoding", "gzip")
+			gzipWriter := gzip.NewWriter(writer)
+			defer gzipWriter.Close()
+			out = gzipWriter
+		}
+		raw, err := io.ReadAll(body)
+		if err != nil {
+			b.Fatalf("read body: %v", err)
+		}
+		var pingRequest ping
+		if err := json.Unmarshal(raw, &pingRequest); err != nil {
+			b.Fatalf("json unmarshal: %v", err)
+		}
+		bs, err := json.Marshal(&pingRequest)
+		if err != nil {
+			b.Fatalf("json marshal: %v", err)
+		}
+		_, err = out.Write(bs)
+		assert.Nil(b, err)
+	}
+
+	server := httptest.NewUnstartedServer(http.HandlerFunc(handler))
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	defer server.Close()
+	twoMiB := strings.Repeat("a", 2*1024*1024)
+	b.ResetTimer()
+
+	b.Run("unary", func(b *testing.B) {
+		b.RunParallel(func(pb *testing.PB) {
+			for pb.Next() {
+				unaryRESTIteration(b, server.Client(), server.URL, twoMiB)
+			}
+		})
+	})
+}
+
+func unaryRESTIteration(b *testing.B, client *http.Client, url string, text string) {
+	b.Helper()
+	rawRequestBody := bytes.NewBuffer(nil)
+	compressedRequestBody := gzip.NewWriter(rawRequestBody)
+	encoder := json.NewEncoder(compressedRequestBody)
+	if err := encoder.Encode(&ping{text}); err != nil {
+		b.Fatalf("marshal request: %v", err)
+	}
+	compressedRequestBody.Close()
+	request, err := http.NewRequestWithContext(
+		context.Background(),
+		http.MethodPost,
+		url,
+		rawRequestBody,
+	)
+	if err != nil {
+		b.Fatalf("construct request: %v", err)
+	}
+	request.Header.Set("Content-Encoding", "gzip")
+	request.Header.Set("Accept-Encoding", "gzip")
+	request.Header.Set("Content-Type", "application/json")
+	response, err := client.Do(request)
+	if err != nil {
+		b.Fatalf("do request: %v", err)
+	}
+	defer func() {
+		_, err := io.Copy(io.Discard, response.Body)
+		assert.Nil(b, err)
+	}()
+	if response.StatusCode != http.StatusOK {
+		b.Fatalf("response status: %v", response.Status)
+	}
+	uncompressed, err := gzip.NewReader(response.Body)
+	if err != nil {
+		b.Fatalf("uncompress response: %v", err)
+	}
+	raw, err := io.ReadAll(uncompressed)
+	if err != nil {
+		b.Fatalf("read response: %v", err)
+	}
+	var got ping
+	if err := json.Unmarshal(raw, &got); err != nil {
+		b.Fatalf("unmarshal: %v", err)
+	}
+}
diff --git a/protocol/triple/triple_protocol/buffer_pool.go b/protocol/triple/triple_protocol/buffer_pool.go
new file mode 100644
index 0000000..495e087
--- /dev/null
+++ b/protocol/triple/triple_protocol/buffer_pool.go
@@ -0,0 +1,54 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"sync"
+)
+
+const (
+	initialBufferSize    = 512
+	maxRecycleBufferSize = 8 * 1024 * 1024 // if >8MiB, don't hold onto a buffer
+)
+
+type bufferPool struct {
+	sync.Pool
+}
+
+func newBufferPool() *bufferPool {
+	return &bufferPool{
+		Pool: sync.Pool{
+			New: func() interface{} {
+				return bytes.NewBuffer(make([]byte, 0, initialBufferSize))
+			},
+		},
+	}
+}
+
+func (b *bufferPool) Get() *bytes.Buffer {
+	if buf, ok := b.Pool.Get().(*bytes.Buffer); ok {
+		return buf
+	}
+	return bytes.NewBuffer(make([]byte, 0, initialBufferSize))
+}
+
+func (b *bufferPool) Put(buffer *bytes.Buffer) {
+	if buffer.Cap() > maxRecycleBufferSize {
+		return
+	}
+	buffer.Reset()
+	b.Pool.Put(buffer)
+}
diff --git a/protocol/triple/triple_protocol/client.go b/protocol/triple/triple_protocol/client.go
new file mode 100644
index 0000000..8e45ba4
--- /dev/null
+++ b/protocol/triple/triple_protocol/client.go
@@ -0,0 +1,265 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"io"
+	"net/http"
+	"net/url"
+	"strings"
+)
+
+// Client is a reusable, concurrency-safe client for a single procedure.
+// Depending on the procedure's type, use the CallUnary, CallClientStream,
+// CallServerStream, or CallBidiStream method.
+//
+// todo:// modify comment
+// By default, clients use the Connect protocol with the binary Protobuf Codec,
+// ask for gzipped responses, and send uncompressed requests. To use the gRPC
+// or gRPC-Web protocols, use the [WithGRPC] or [WithGRPCWeb] options.
+type Client struct {
+	config         *clientConfig
+	callUnary      func(context.Context, *Request, *Response) error
+	protocolClient protocolClient
+	err            error
+}
+
+// NewClient constructs a new Client.
+func NewClient(httpClient HTTPClient, url string, options ...ClientOption) *Client {
+	client := &Client{}
+	config, err := newClientConfig(url, options)
+	if err != nil {
+		client.err = err
+		return client
+	}
+	client.config = config
+	protocolClient, protocolErr := client.config.Protocol.NewClient(
+		&protocolClientParams{
+			CompressionName: config.RequestCompressionName,
+			CompressionPools: newReadOnlyCompressionPools(
+				config.CompressionPools,
+				config.CompressionNames,
+			),
+			Codec:            config.Codec,
+			Protobuf:         config.protobuf(),
+			CompressMinBytes: config.CompressMinBytes,
+			HTTPClient:       httpClient,
+			URL:              config.URL,
+			BufferPool:       config.BufferPool,
+			ReadMaxBytes:     config.ReadMaxBytes,
+			SendMaxBytes:     config.SendMaxBytes,
+			EnableGet:        config.EnableGet,
+			GetURLMaxBytes:   config.GetURLMaxBytes,
+			GetUseFallback:   config.GetUseFallback,
+		},
+	)
+	if protocolErr != nil {
+		client.err = protocolErr
+		return client
+	}
+	client.protocolClient = protocolClient
+	// Rather than applying unary interceptors along the hot path, we can do it
+	// once at client creation.
+	unarySpec := config.newSpec(StreamTypeUnary)
+	unaryFunc := UnaryFunc(func(ctx context.Context, request AnyRequest, response AnyResponse) error {
+		conn := client.protocolClient.NewConn(ctx, unarySpec, request.Header())
+		// Send always returns an io.EOF unless the error is from the client-side.
+		// We want the user to continue to call Receive in those cases to get the
+		// full error from the server-side.
+		if err := conn.Send(request.Any()); err != nil && !errors.Is(err, io.EOF) {
+			_ = conn.CloseRequest()
+			_ = conn.CloseResponse()
+			return err
+		}
+		if err := conn.CloseRequest(); err != nil {
+			_ = conn.CloseResponse()
+			return err
+		}
+		if err := receiveUnaryResponse(conn, response); err != nil {
+			_ = conn.CloseResponse()
+			return err
+		}
+		return conn.CloseResponse()
+	})
+	if interceptor := config.Interceptor; interceptor != nil {
+		unaryFunc = interceptor.WrapUnary(unaryFunc)
+	}
+	client.callUnary = func(ctx context.Context, request *Request, response *Response) error {
+		// To make the specification, peer, and RPC headers visible to the full
+		// interceptor chain (as though they were supplied by the caller), we'll
+		// add them here.
+		request.spec = unarySpec
+		request.peer = client.protocolClient.Peer()
+		protocolClient.WriteRequestHeader(StreamTypeUnary, request.Header())
+		if err := unaryFunc(ctx, request, response); err != nil {
+			return err
+		}
+		//typed, ok := response.(*Response[Res])
+		//if !ok {
+		//	return nil, errorf(CodeInternal, "unexpected client response type %T", response)
+		//}
+		return nil
+	}
+	return client
+}
+
+// CallUnary calls a request-response procedure.
+func (c *Client) CallUnary(ctx context.Context, request *Request, response *Response) error {
+	if c.err != nil {
+		return c.err
+	}
+	return c.callUnary(ctx, request, response)
+}
+
+// CallClientStream calls a client streaming procedure.
+func (c *Client) CallClientStream(ctx context.Context) (*ClientStreamForClient, error) {
+	if c.err != nil {
+		return &ClientStreamForClient{err: c.err}, c.err
+	}
+	return &ClientStreamForClient{conn: c.newConn(ctx, StreamTypeClient)}, nil
+}
+
+// CallServerStream calls a server streaming procedure.
+func (c *Client) CallServerStream(ctx context.Context, request *Request) (*ServerStreamForClient, error) {
+	if c.err != nil {
+		return nil, c.err
+	}
+	conn := c.newConn(ctx, StreamTypeServer)
+	request.spec = conn.Spec()
+	request.peer = conn.Peer()
+	mergeHeaders(conn.RequestHeader(), request.header)
+	// Send always returns an io.EOF unless the error is from the client-side.
+	// We want the user to continue to call Receive in those cases to get the
+	// full error from the server-side.
+	if err := conn.Send(request.Msg); err != nil && !errors.Is(err, io.EOF) {
+		_ = conn.CloseRequest()
+		_ = conn.CloseResponse()
+		return nil, err
+	}
+	if err := conn.CloseRequest(); err != nil {
+		return nil, err
+	}
+	return &ServerStreamForClient{conn: conn}, nil
+}
+
+// CallBidiStream calls a bidirectional streaming procedure.
+func (c *Client) CallBidiStream(ctx context.Context) (*BidiStreamForClient, error) {
+	if c.err != nil {
+		return &BidiStreamForClient{err: c.err}, c.err
+	}
+	return &BidiStreamForClient{conn: c.newConn(ctx, StreamTypeBidi)}, nil
+}
+
+func (c *Client) newConn(ctx context.Context, streamType StreamType) StreamingClientConn {
+	newConn := func(ctx context.Context, spec Spec) StreamingClientConn {
+		header := make(http.Header, 8) // arbitrary power of two, prevent immediate resizing
+		c.protocolClient.WriteRequestHeader(streamType, header)
+		return c.protocolClient.NewConn(ctx, spec, header)
+	}
+	if interceptor := c.config.Interceptor; interceptor != nil {
+		newConn = interceptor.WrapStreamingClient(newConn)
+	}
+	return newConn(ctx, c.config.newSpec(streamType))
+}
+
+type clientConfig struct {
+	URL                    *url.URL
+	Protocol               protocol
+	Procedure              string
+	CompressMinBytes       int
+	Interceptor            Interceptor
+	CompressionPools       map[string]*compressionPool
+	CompressionNames       []string
+	Codec                  Codec
+	RequestCompressionName string
+	BufferPool             *bufferPool
+	ReadMaxBytes           int
+	SendMaxBytes           int
+	EnableGet              bool
+	GetURLMaxBytes         int
+	GetUseFallback         bool
+	IdempotencyLevel       IdempotencyLevel
+}
+
+func newClientConfig(rawURL string, options []ClientOption) (*clientConfig, *Error) {
+	url, err := parseRequestURL(rawURL)
+	if err != nil {
+		return nil, err
+	}
+	protoPath := extractProtoPath(url.Path)
+	config := clientConfig{
+		URL:              url,
+		Protocol:         &protocolGRPC{},
+		Procedure:        protoPath,
+		CompressionPools: make(map[string]*compressionPool),
+		BufferPool:       newBufferPool(),
+	}
+	withProtoBinaryCodec().applyToClient(&config)
+	withGzip().applyToClient(&config)
+	for _, opt := range options {
+		opt.applyToClient(&config)
+	}
+	if err := config.validate(); err != nil {
+		return nil, err
+	}
+	return &config, nil
+}
+
+func (c *clientConfig) validate() *Error {
+	if c.Codec == nil || c.Codec.Name() == "" {
+		return errorf(CodeUnknown, "no codec configured")
+	}
+	if c.RequestCompressionName != "" && c.RequestCompressionName != compressionIdentity {
+		if _, ok := c.CompressionPools[c.RequestCompressionName]; !ok {
+			return errorf(CodeUnknown, "unknown compression %q", c.RequestCompressionName)
+		}
+	}
+	return nil
+}
+
+func (c *clientConfig) protobuf() Codec {
+	if c.Codec.Name() == codecNameProto {
+		return c.Codec
+	}
+	return &protoBinaryCodec{}
+}
+
+func (c *clientConfig) newSpec(t StreamType) Spec {
+	return Spec{
+		StreamType:       t,
+		Procedure:        c.Procedure,
+		IsClient:         true,
+		IdempotencyLevel: c.IdempotencyLevel,
+	}
+}
+
+func parseRequestURL(rawURL string) (*url.URL, *Error) {
+	url, err := url.ParseRequestURI(rawURL)
+	if err == nil {
+		return url, nil
+	}
+	if !strings.Contains(rawURL, "://") {
+		// URL doesn't have a scheme, so the user is likely accustomed to
+		// grpc-go's APIs.
+		err = fmt.Errorf(
+			"URL %q missing scheme: use http:// or https:// (unlike grpc-go)",
+			rawURL,
+		)
+	}
+	return nil, NewError(CodeUnavailable, err)
+}
diff --git a/protocol/triple/triple_protocol/client_example_test.go b/protocol/triple/triple_protocol/client_example_test.go
new file mode 100644
index 0000000..98433e6
--- /dev/null
+++ b/protocol/triple/triple_protocol/client_example_test.go
@@ -0,0 +1,58 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"log"
+	"os"
+)
+
+import (
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func Example_client() {
+	logger := log.New(os.Stdout, "" /* prefix */, 0 /* flags */)
+	// Unfortunately, pkg.go.dev can't run examples that actually use the
+	// network. To keep this example runnable, we'll use an HTTP server and
+	// client that communicate over in-memory pipes. The client is still a plain
+	// *http.Client!
+	httpClient := examplePingServer.Client()
+
+	// By default, clients use the GRPC protocol. Add triple_protocol.WithTriple() or
+	// triple_protocol.WithGRPCWeb() to switch protocols.
+	client := pingv1connect.NewPingServiceClient(
+		httpClient,
+		examplePingServer.URL(),
+	)
+	response := tri.NewResponse(&pingv1.PingResponse{})
+	if err := client.Ping(
+		context.Background(),
+		tri.NewRequest(&pingv1.PingRequest{Number: 42}),
+		response,
+	); err != nil {
+		logger.Println("error:", err)
+		return
+	}
+	logger.Println("response content-type:", response.Header().Get("Content-Type"))
+	logger.Println("response message:", response.Msg)
+
+	// Output:
+	// response content-type: application/grpc+proto
+	// response message: number:42
+}
diff --git a/protocol/triple/triple_protocol/client_ext_test.go b/protocol/triple/triple_protocol/client_ext_test.go
new file mode 100644
index 0000000..2bf8080
--- /dev/null
+++ b/protocol/triple/triple_protocol/client_ext_test.go
@@ -0,0 +1,186 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"errors"
+	"net/http"
+	"net/http/httptest"
+	"strings"
+	"testing"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func TestNewClient_InitFailure(t *testing.T) {
+	t.Parallel()
+	client := pingv1connect.NewPingServiceClient(
+		http.DefaultClient,
+		"http://127.0.0.1:8080",
+		// This triggers an error during initialization, so each call will short circuit returning an error.
+		triple.WithSendCompression("invalid"),
+	)
+	validateExpectedError := func(t *testing.T, err error) {
+		t.Helper()
+		assert.NotNil(t, err)
+		var tripleErr *triple.Error
+		assert.True(t, errors.As(err, &tripleErr))
+		assert.Equal(t, tripleErr.Message(), `unknown compression "invalid"`)
+	}
+
+	t.Run("unary", func(t *testing.T) {
+		t.Parallel()
+		err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+		validateExpectedError(t, err)
+	})
+
+	t.Run("bidi", func(t *testing.T) {
+		t.Parallel()
+		bidiStream, err := client.CumSum(context.Background())
+		validateExpectedError(t, err)
+		err = bidiStream.Send(&pingv1.CumSumRequest{})
+		validateExpectedError(t, err)
+	})
+
+	t.Run("client_stream", func(t *testing.T) {
+		t.Parallel()
+		clientStream, err := client.Sum(context.Background())
+		validateExpectedError(t, err)
+		err = clientStream.Send(&pingv1.SumRequest{})
+		validateExpectedError(t, err)
+	})
+
+	t.Run("server_stream", func(t *testing.T) {
+		t.Parallel()
+		_, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{Number: 3}))
+		validateExpectedError(t, err)
+	})
+}
+
+func TestClientPeer(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+
+	run := func(t *testing.T, streamFlag bool, opts ...triple.ClientOption) {
+		t.Helper()
+		client := pingv1connect.NewPingServiceClient(
+			server.Client(),
+			server.URL,
+			triple.WithClientOptions(opts...),
+			triple.WithInterceptors(&assertPeerInterceptor{t}),
+		)
+		ctx := context.Background()
+		// unary
+		err := client.Ping(ctx, triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+		assert.Nil(t, err)
+		text := strings.Repeat(".", 256)
+		resp := &pingv1.PingResponse{}
+		err = client.Ping(ctx, triple.NewRequest(&pingv1.PingRequest{Text: text}), triple.NewResponse(resp))
+		assert.Nil(t, err)
+		assert.Equal(t, resp.Text, text)
+		// protocol does not support stream just need to test unary
+		if !streamFlag {
+			return
+		}
+
+		//client streaming
+		clientStream, err := client.Sum(ctx)
+		assert.Nil(t, err)
+		t.Cleanup(func() {
+			closeErr := clientStream.CloseAndReceive(triple.NewResponse(&pingv1.SumResponse{}))
+			assert.Nil(t, closeErr)
+		})
+		assert.NotZero(t, clientStream.Peer().Addr)
+		assert.NotZero(t, clientStream.Peer().Protocol)
+		err = clientStream.Send(&pingv1.SumRequest{})
+		assert.Nil(t, err)
+		//server streaming
+		serverStream, err := client.CountUp(ctx, triple.NewRequest(&pingv1.CountUpRequest{}))
+		t.Cleanup(func() {
+			assert.Nil(t, serverStream.Close())
+		})
+		assert.Nil(t, err)
+		// bidi streaming
+		bidiStream, err := client.CumSum(ctx)
+		assert.Nil(t, err)
+		t.Cleanup(func() {
+			assert.Nil(t, bidiStream.CloseRequest())
+			assert.Nil(t, bidiStream.CloseResponse())
+		})
+		assert.NotZero(t, bidiStream.Peer().Addr)
+		assert.NotZero(t, bidiStream.Peer().Protocol)
+		err = bidiStream.Send(&pingv1.CumSumRequest{})
+		assert.Nil(t, err)
+	}
+
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		run(t /*streamFlag*/, false, triple.WithTriple())
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		run(t, true)
+	})
+}
+
+type assertPeerInterceptor struct {
+	tb testing.TB
+}
+
+func (a *assertPeerInterceptor) WrapUnaryHandler(next triple.UnaryHandlerFunc) triple.UnaryHandlerFunc {
+	return func(ctx context.Context, req triple.AnyRequest) (triple.AnyResponse, error) {
+		assert.NotZero(a.tb, req.Peer().Addr)
+		assert.NotZero(a.tb, req.Peer().Protocol)
+		return next(ctx, req)
+	}
+}
+
+func (a *assertPeerInterceptor) WrapUnary(next triple.UnaryFunc) triple.UnaryFunc {
+	return func(ctx context.Context, req triple.AnyRequest, res triple.AnyResponse) error {
+		assert.NotZero(a.tb, req.Peer().Addr)
+		assert.NotZero(a.tb, req.Peer().Protocol)
+		return next(ctx, req, res)
+	}
+}
+
+func (a *assertPeerInterceptor) WrapStreamingClient(next triple.StreamingClientFunc) triple.StreamingClientFunc {
+	return func(ctx context.Context, spec triple.Spec) triple.StreamingClientConn {
+		conn := next(ctx, spec)
+		assert.NotZero(a.tb, conn.Peer().Addr)
+		assert.NotZero(a.tb, conn.Peer().Protocol)
+		assert.NotZero(a.tb, conn.Spec())
+		return conn
+	}
+}
+
+func (a *assertPeerInterceptor) WrapStreamingHandler(next triple.StreamingHandlerFunc) triple.StreamingHandlerFunc {
+	return func(ctx context.Context, conn triple.StreamingHandlerConn) error {
+		assert.NotZero(a.tb, conn.Peer().Addr)
+		assert.NotZero(a.tb, conn.Peer().Protocol)
+		assert.NotZero(a.tb, conn.Spec())
+		return next(ctx, conn)
+	}
+}
diff --git a/protocol/triple/triple_protocol/client_stream.go b/protocol/triple/triple_protocol/client_stream.go
new file mode 100644
index 0000000..0f01f29
--- /dev/null
+++ b/protocol/triple/triple_protocol/client_stream.go
@@ -0,0 +1,272 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"errors"
+	"io"
+	"net/http"
+)
+
+// ClientStreamForClient is the client's view of a client streaming RPC.
+//
+// It's returned from [Client].CallClientStream, but doesn't currently have an
+// exported constructor function.
+type ClientStreamForClient struct {
+	conn StreamingClientConn
+	// Error from client construction. If non-nil, return for all calls.
+	err error
+}
+
+// Spec returns the specification for the RPC.
+func (c *ClientStreamForClient) Spec() Spec {
+	return c.conn.Spec()
+}
+
+// Peer describes the server for the RPC.
+func (c *ClientStreamForClient) Peer() Peer {
+	return c.conn.Peer()
+}
+
+// RequestHeader returns the request headers. Headers are sent to the server with the
+// first call to Send.
+//
+// Headers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (c *ClientStreamForClient) RequestHeader() http.Header {
+	if c.err != nil {
+		return http.Header{}
+	}
+	return c.conn.RequestHeader()
+}
+
+// Send a message to the server. The first call to Send also sends the request
+// headers.
+//
+// If the server returns an error, Send returns an error that wraps [io.EOF].
+// Clients should check for case using the standard library's [errors.Is] and
+// unmarshal the error using CloseAndReceive.
+func (c *ClientStreamForClient) Send(request interface{}) error {
+	if c.err != nil {
+		return c.err
+	}
+	if request == nil {
+		return c.conn.Send(nil)
+	}
+	return c.conn.Send(request)
+}
+
+// CloseAndReceive closes the send side of the stream and waits for the
+// response.
+func (c *ClientStreamForClient) CloseAndReceive(response *Response) error {
+	if c.err != nil {
+		return c.err
+	}
+	if err := c.conn.CloseRequest(); err != nil {
+		_ = c.conn.CloseResponse()
+		return err
+	}
+	if err := receiveUnaryResponse(c.conn, response); err != nil {
+		_ = c.conn.CloseResponse()
+		return err
+	}
+	return c.conn.CloseResponse()
+}
+
+// Conn exposes the underlying StreamingClientConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (c *ClientStreamForClient) Conn() (StreamingClientConn, error) {
+	return c.conn, c.err
+}
+
+// ServerStreamForClient is the client's view of a server streaming RPC.
+//
+// It's returned from [Client].CallServerStream, but doesn't currently have an
+// exported constructor function.
+type ServerStreamForClient struct {
+	conn StreamingClientConn
+	msg  interface{}
+	// Error from client construction. If non-nil, return for all calls.
+	constructErr error
+	// Error from conn.Receive().
+	receiveErr error
+}
+
+// Receive advances the stream to the next message, which will then be
+// available through the Msg method. It returns false when the stream stops,
+// either by reaching the end or by encountering an unexpected error. After
+// Receive returns false, the Err method will return any unexpected error
+// encountered.
+func (s *ServerStreamForClient) Receive(msg interface{}) bool {
+	if s.constructErr != nil || s.receiveErr != nil {
+		return false
+	}
+	s.msg = msg
+	s.receiveErr = s.conn.Receive(s.msg)
+	return s.receiveErr == nil
+}
+
+// Msg returns the most recent message unmarshaled by a call to Receive.
+func (s *ServerStreamForClient) Msg() interface{} {
+	// todo:// processing nil pointer
+	//if s.msg == nil {
+	//	s.msg = new(Res)
+	//}
+	return s.msg
+}
+
+// Err returns the first non-EOF error that was encountered by Receive.
+func (s *ServerStreamForClient) Err() error {
+	if s.constructErr != nil {
+		return s.constructErr
+	}
+	if s.receiveErr != nil && !errors.Is(s.receiveErr, io.EOF) {
+		return s.receiveErr
+	}
+	return nil
+}
+
+// ResponseHeader returns the headers received from the server. It blocks until
+// the first call to Receive returns.
+func (s *ServerStreamForClient) ResponseHeader() http.Header {
+	if s.constructErr != nil {
+		return http.Header{}
+	}
+	return s.conn.ResponseHeader()
+}
+
+// ResponseTrailer returns the trailers received from the server. Trailers
+// aren't fully populated until Receive() returns an error wrapping io.EOF.
+func (s *ServerStreamForClient) ResponseTrailer() http.Header {
+	if s.constructErr != nil {
+		return http.Header{}
+	}
+	return s.conn.ResponseTrailer()
+}
+
+// Close the receive side of the stream.
+func (s *ServerStreamForClient) Close() error {
+	if s.constructErr != nil {
+		return s.constructErr
+	}
+	return s.conn.CloseResponse()
+}
+
+// Conn exposes the underlying StreamingClientConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (s *ServerStreamForClient) Conn() (StreamingClientConn, error) {
+	return s.conn, s.constructErr
+}
+
+// BidiStreamForClient is the client's view of a bidirectional streaming RPC.
+//
+// It's returned from [Client].CallBidiStream, but doesn't currently have an
+// exported constructor function.
+type BidiStreamForClient struct {
+	conn StreamingClientConn
+	// Error from client construction. If non-nil, return for all calls.
+	err error
+}
+
+// Spec returns the specification for the RPC.
+func (b *BidiStreamForClient) Spec() Spec {
+	return b.conn.Spec()
+}
+
+// Peer describes the server for the RPC.
+func (b *BidiStreamForClient) Peer() Peer {
+	return b.conn.Peer()
+}
+
+// RequestHeader returns the request headers. Headers are sent with the first
+// call to Send.
+//
+// Headers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (b *BidiStreamForClient) RequestHeader() http.Header {
+	if b.err != nil {
+		return http.Header{}
+	}
+	return b.conn.RequestHeader()
+}
+
+// Send a message to the server. The first call to Send also sends the request
+// headers. To send just the request headers, without a body, call Send with a
+// nil pointer.
+//
+// If the server returns an error, Send returns an error that wraps [io.EOF].
+// Clients should check for EOF using the standard library's [errors.Is] and
+// call Receive to retrieve the error.
+func (b *BidiStreamForClient) Send(msg interface{}) error {
+	if b.err != nil {
+		return b.err
+	}
+	if msg == nil {
+		return b.conn.Send(nil)
+	}
+	return b.conn.Send(msg)
+}
+
+// CloseRequest closes the send side of the stream.
+func (b *BidiStreamForClient) CloseRequest() error {
+	if b.err != nil {
+		return b.err
+	}
+	return b.conn.CloseRequest()
+}
+
+// Receive a message. When the server is done sending messages and no other
+// errors have occurred, Receive will return an error that wraps [io.EOF].
+func (b *BidiStreamForClient) Receive(msg interface{}) error {
+	if b.err != nil {
+		return b.err
+	}
+	if err := b.conn.Receive(msg); err != nil {
+		return err
+	}
+	return nil
+}
+
+// CloseResponse closes the receive side of the stream.
+func (b *BidiStreamForClient) CloseResponse() error {
+	if b.err != nil {
+		return b.err
+	}
+	return b.conn.CloseResponse()
+}
+
+// ResponseHeader returns the headers received from the server. It blocks until
+// the first call to Receive returns.
+func (b *BidiStreamForClient) ResponseHeader() http.Header {
+	if b.err != nil {
+		return http.Header{}
+	}
+	return b.conn.ResponseHeader()
+}
+
+// ResponseTrailer returns the trailers received from the server. Trailers
+// aren't fully populated until Receive() returns an error wrapping [io.EOF].
+func (b *BidiStreamForClient) ResponseTrailer() http.Header {
+	if b.err != nil {
+		return http.Header{}
+	}
+	return b.conn.ResponseTrailer()
+}
+
+// Conn exposes the underlying StreamingClientConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (b *BidiStreamForClient) Conn() (StreamingClientConn, error) {
+	return b.conn, b.err
+}
diff --git a/protocol/triple/triple_protocol/client_stream_test.go b/protocol/triple/triple_protocol/client_stream_test.go
new file mode 100644
index 0000000..2b959f8
--- /dev/null
+++ b/protocol/triple/triple_protocol/client_stream_test.go
@@ -0,0 +1,106 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+	"testing"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+)
+
+func TestClientStreamForClient_NoPanics(t *testing.T) {
+	t.Parallel()
+	initErr := errors.New("client init failure")
+	clientStream := &ClientStreamForClient{err: initErr}
+	assert.ErrorIs(t, clientStream.Send(&pingv1.PingRequest{}), initErr)
+	verifyHeaders(t, clientStream.RequestHeader())
+	res := NewResponse(&pingv1.PingResponse{})
+	err := clientStream.CloseAndReceive(res)
+	assert.ErrorIs(t, err, initErr)
+	conn, err := clientStream.Conn()
+	assert.NotNil(t, err)
+	assert.Nil(t, conn)
+}
+
+func TestServerStreamForClient_NoPanics(t *testing.T) {
+	t.Parallel()
+	initErr := errors.New("client init failure")
+	serverStream := &ServerStreamForClient{constructErr: initErr}
+	assert.ErrorIs(t, serverStream.Err(), initErr)
+	assert.ErrorIs(t, serverStream.Close(), initErr)
+	assert.Nil(t, serverStream.Msg())
+	assert.False(t, serverStream.Receive(NewResponse(&pingv1.PingResponse{})))
+	verifyHeaders(t, serverStream.ResponseHeader())
+	verifyHeaders(t, serverStream.ResponseTrailer())
+	conn, err := serverStream.Conn()
+	assert.NotNil(t, err)
+	assert.Nil(t, conn)
+}
+
+func TestServerStreamForClient(t *testing.T) {
+	t.Parallel()
+	stream := &ServerStreamForClient{conn: &nopStreamingClientConn{}}
+	res := NewResponse(&pingv1.PingResponse{})
+	assert.True(t, stream.Receive(res))
+	first := fmt.Sprintf("%p", stream.Msg())
+	res1 := NewResponse(&pingv1.PingResponse{})
+	assert.True(t, stream.Receive(res1))
+	second := fmt.Sprintf("%p", stream.Msg())
+	assert.NotEqual(t, first, second)
+	conn, err := stream.Conn()
+	assert.Nil(t, err)
+	assert.NotNil(t, conn)
+}
+
+func TestBidiStreamForClient_NoPanics(t *testing.T) {
+	t.Parallel()
+	initErr := errors.New("client init failure")
+	bidiStream := &BidiStreamForClient{err: initErr}
+	res := NewResponse(&pingv1.CumSumResponse{})
+	err := bidiStream.Receive(res)
+	assert.ErrorIs(t, err, initErr)
+	verifyHeaders(t, bidiStream.RequestHeader())
+	verifyHeaders(t, bidiStream.ResponseHeader())
+	verifyHeaders(t, bidiStream.ResponseTrailer())
+	assert.ErrorIs(t, bidiStream.Send(&pingv1.CumSumRequest{}), initErr)
+	assert.ErrorIs(t, bidiStream.CloseRequest(), initErr)
+	assert.ErrorIs(t, bidiStream.CloseResponse(), initErr)
+	conn, err := bidiStream.Conn()
+	assert.NotNil(t, err)
+	assert.Nil(t, conn)
+}
+
+func verifyHeaders(t *testing.T, headers http.Header) {
+	t.Helper()
+	assert.Equal(t, headers, http.Header{})
+
+	// Verify set/del don't panic
+	headers.Set("a", "b")
+	headers.Del("a")
+}
+
+type nopStreamingClientConn struct {
+	StreamingClientConn
+}
+
+func (c *nopStreamingClientConn) Receive(msg interface{}) error {
+	return nil
+}
diff --git a/protocol/triple/triple_protocol/code.go b/protocol/triple/triple_protocol/code.go
new file mode 100644
index 0000000..95d6345
--- /dev/null
+++ b/protocol/triple/triple_protocol/code.go
@@ -0,0 +1,226 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"fmt"
+	"strconv"
+	"strings"
+)
+
+// A Code is one of the Connect protocol's error codes. There are no user-defined
+// codes, so only the codes enumerated below are valid. In both name and
+// semantics, these codes match the gRPC status codes.
+//
+// The descriptions below are optimized for brevity rather than completeness.
+// See the [Connect protocol specification] for detailed descriptions of each
+// code and example usage.
+//
+// [Connect protocol specification]: https://connect.build/docs/protocol
+type Code uint32
+
+const (
+	// The zero code in gRPC is OK, which indicates that the operation was a
+	// success. We don't define a constant for it because it overlaps awkwardly
+	// with Go's error semantics: what does it mean to have a non-nil error with
+	// an OK status? (Also, the Connect protocol doesn't use a code for
+	// successes.)
+
+	// CodeCanceled indicates that the operation was canceled, typically by the
+	// caller.
+	CodeCanceled Code = 1
+
+	// CodeUnknown indicates that the operation failed for an unknown reason.
+	CodeUnknown Code = 2
+
+	// CodeInvalidArgument indicates that client supplied an invalid argument.
+	CodeInvalidArgument Code = 3
+
+	// CodeDeadlineExceeded indicates that deadline expired before the operation
+	// could complete.
+	CodeDeadlineExceeded Code = 4
+
+	// CodeNotFound indicates that some requested entity (for example, a file or
+	// directory) was not found.
+	CodeNotFound Code = 5
+
+	// CodeAlreadyExists indicates that client attempted to create an entity (for
+	// example, a file or directory) that already exists.
+	CodeAlreadyExists Code = 6
+
+	// CodePermissionDenied indicates that the caller doesn't have permission to
+	// execute the specified operation.
+	CodePermissionDenied Code = 7
+
+	// CodeResourceExhausted indicates that some resource has been exhausted. For
+	// example, a per-user quota may be exhausted or the entire file system may
+	// be full.
+	CodeResourceExhausted Code = 8
+
+	// CodeFailedPrecondition indicates that the system is not in a state
+	// required for the operation's execution.
+	CodeFailedPrecondition Code = 9
+
+	// CodeAborted indicates that operation was aborted by the system, usually
+	// because of a concurrency issue such as a sequencer check failure or
+	// transaction abort.
+	CodeAborted Code = 10
+
+	// CodeOutOfRange indicates that the operation was attempted past the valid
+	// range (for example, seeking past end-of-file).
+	CodeOutOfRange Code = 11
+
+	// CodeUnimplemented indicates that the operation isn't implemented,
+	// supported, or enabled in this service.
+	CodeUnimplemented Code = 12
+
+	// CodeInternal indicates that some invariants expected by the underlying
+	// system have been broken. This code is reserved for serious errors.
+	CodeInternal Code = 13
+
+	// CodeUnavailable indicates that the service is currently unavailable. This
+	// is usually temporary, so clients can back off and retry idempotent
+	// operations.
+	CodeUnavailable Code = 14
+
+	// CodeDataLoss indicates that the operation has resulted in unrecoverable
+	// data loss or corruption.
+	CodeDataLoss Code = 15
+
+	// CodeUnauthenticated indicates that the request does not have valid
+	// authentication credentials for the operation.
+	CodeUnauthenticated Code = 16
+
+	minCode = CodeCanceled
+	maxCode = CodeUnauthenticated
+)
+
+func (c Code) String() string {
+	switch c {
+	case CodeCanceled:
+		return "canceled"
+	case CodeUnknown:
+		return "unknown"
+	case CodeInvalidArgument:
+		return "invalid_argument"
+	case CodeDeadlineExceeded:
+		return "deadline_exceeded"
+	case CodeNotFound:
+		return "not_found"
+	case CodeAlreadyExists:
+		return "already_exists"
+	case CodePermissionDenied:
+		return "permission_denied"
+	case CodeResourceExhausted:
+		return "resource_exhausted"
+	case CodeFailedPrecondition:
+		return "failed_precondition"
+	case CodeAborted:
+		return "aborted"
+	case CodeOutOfRange:
+		return "out_of_range"
+	case CodeUnimplemented:
+		return "unimplemented"
+	case CodeInternal:
+		return "internal"
+	case CodeUnavailable:
+		return "unavailable"
+	case CodeDataLoss:
+		return "data_loss"
+	case CodeUnauthenticated:
+		return "unauthenticated"
+	}
+	return fmt.Sprintf("code_%d", c)
+}
+
+// MarshalText implements [encoding.TextMarshaler].
+func (c Code) MarshalText() ([]byte, error) {
+	return []byte(c.String()), nil
+}
+
+// UnmarshalText implements [encoding.TextUnmarshaler].
+func (c *Code) UnmarshalText(data []byte) error {
+	dataStr := string(data)
+	switch dataStr {
+	case "canceled":
+		*c = CodeCanceled
+		return nil
+	case "unknown":
+		*c = CodeUnknown
+		return nil
+	case "invalid_argument":
+		*c = CodeInvalidArgument
+		return nil
+	case "deadline_exceeded":
+		*c = CodeDeadlineExceeded
+		return nil
+	case "not_found":
+		*c = CodeNotFound
+		return nil
+	case "already_exists":
+		*c = CodeAlreadyExists
+		return nil
+	case "permission_denied":
+		*c = CodePermissionDenied
+		return nil
+	case "resource_exhausted":
+		*c = CodeResourceExhausted
+		return nil
+	case "failed_precondition":
+		*c = CodeFailedPrecondition
+		return nil
+	case "aborted":
+		*c = CodeAborted
+		return nil
+	case "out_of_range":
+		*c = CodeOutOfRange
+		return nil
+	case "unimplemented":
+		*c = CodeUnimplemented
+		return nil
+	case "internal":
+		*c = CodeInternal
+		return nil
+	case "unavailable":
+		*c = CodeUnavailable
+		return nil
+	case "data_loss":
+		*c = CodeDataLoss
+		return nil
+	case "unauthenticated":
+		*c = CodeUnauthenticated
+		return nil
+	}
+	// Ensure that non-canonical codes round-trip through MarshalText and
+	// UnmarshalText.
+	if strings.HasPrefix(dataStr, "code_") {
+		dataStr = strings.TrimPrefix(dataStr, "code_")
+		code, err := strconv.ParseUint(dataStr, 10 /* base */, 64 /* bitsize */)
+		if err == nil && (code < uint64(minCode) || code > uint64(maxCode)) {
+			*c = Code(code)
+			return nil
+		}
+	}
+	return fmt.Errorf("invalid code %q", dataStr)
+}
+
+// CodeOf returns the error's status code if it is or wraps an [*Error] and
+// [CodeUnknown] otherwise.
+func CodeOf(err error) Code {
+	if connectErr, ok := asError(err); ok {
+		return connectErr.Code()
+	}
+	return CodeUnknown
+}
diff --git a/protocol/triple/triple_protocol/code_test.go b/protocol/triple/triple_protocol/code_test.go
new file mode 100644
index 0000000..43bc5ca
--- /dev/null
+++ b/protocol/triple/triple_protocol/code_test.go
@@ -0,0 +1,58 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"strconv"
+	"strings"
+	"testing"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestCode(t *testing.T) {
+	t.Parallel()
+	var valid []Code
+	for code := minCode; code <= maxCode; code++ {
+		valid = append(valid, code)
+	}
+	// Ensures that we don't forget to update the mapping in the Stringer
+	// implementation.
+	for _, code := range valid {
+		assert.False(
+			t,
+			strings.HasPrefix(code.String(), "code_"),
+			assert.Sprintf("update Code.String() method for new code %v", code),
+		)
+		assertCodeRoundTrips(t, code)
+	}
+	assertCodeRoundTrips(t, Code(999))
+}
+
+func assertCodeRoundTrips(tb testing.TB, code Code) {
+	tb.Helper()
+	encoded, err := code.MarshalText()
+	assert.Nil(tb, err)
+	var decoded Code
+	assert.Nil(tb, decoded.UnmarshalText(encoded))
+	assert.Equal(tb, decoded, code)
+	if code >= minCode && code <= maxCode {
+		var invalid Code
+		// For the known codes, we only accept the canonical string representation: "canceled", not "code_1".
+		assert.NotNil(tb, invalid.UnmarshalText([]byte("code_"+strconv.Itoa(int(code)))))
+	}
+}
diff --git a/protocol/triple/triple_protocol/codec.go b/protocol/triple/triple_protocol/codec.go
new file mode 100644
index 0000000..cc33e3c
--- /dev/null
+++ b/protocol/triple/triple_protocol/codec.go
@@ -0,0 +1,226 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"encoding/json"
+	"errors"
+	"fmt"
+)
+
+import (
+	"google.golang.org/protobuf/encoding/protojson"
+
+	"google.golang.org/protobuf/proto"
+
+	"google.golang.org/protobuf/runtime/protoiface"
+)
+
+const (
+	codecNameProto           = "proto"
+	codecNameJSON            = "json"
+	codecNameJSONCharsetUTF8 = codecNameJSON + "; charset=utf-8"
+)
+
+// Codec marshals structs (typically generated from a schema) to and from bytes.
+type Codec interface {
+	// Name returns the name of the Codec.
+	//
+	// This may be used as part of the Content-Type within HTTP. For example,
+	// with gRPC this is the content subtype, so "application/grpc+proto" will
+	// map to the Codec with name "proto".
+	//
+	// Names must not be empty.
+	Name() string
+	// Marshal marshals the given message.
+	//
+	// Marshal may expect a specific type of message, and will error if this type
+	// is not given.
+	Marshal(interface{}) ([]byte, error)
+	// Unmarshal unmarshals the given message.
+	//
+	// Unmarshal may expect a specific type of message, and will error if this
+	// type is not given.
+	Unmarshal([]byte, interface{}) error
+}
+
+// stableCodec is an extension to Codec for serializing with stable output.
+type stableCodec interface {
+	Codec
+
+	// MarshalStable marshals the given message with stable field ordering.
+	//
+	// MarshalStable should return the same output for a given input. Although
+	// it is not guaranteed to be canonicalized, the marshalling routine for
+	// MarshalStable will opt for the most normalized output available for a
+	// given serialization.
+	//
+	// For practical reasons, it is possible for MarshalStable to return two
+	// different results for two inputs considered to be "equal" in their own
+	// domain, and it may change in the future with codec updates, but for
+	// any given concrete value and any given version, it should return the
+	// same output.
+	MarshalStable(interface{}) ([]byte, error)
+
+	// IsBinary returns true if the marshalled data is binary for this codec.
+	//
+	// If this function returns false, the data returned from Marshal and
+	// MarshalStable are considered valid text and may be used in contexts
+	// where text is expected.
+	IsBinary() bool
+}
+
+type protoBinaryCodec struct{}
+
+var _ Codec = (*protoBinaryCodec)(nil)
+
+func (c *protoBinaryCodec) Name() string { return codecNameProto }
+
+func (c *protoBinaryCodec) Marshal(message interface{}) ([]byte, error) {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return nil, errNotProto(message)
+	}
+	return proto.Marshal(protoMessage)
+}
+
+func (c *protoBinaryCodec) Unmarshal(data []byte, message interface{}) error {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return errNotProto(message)
+	}
+	return proto.Unmarshal(data, protoMessage)
+}
+
+func (c *protoBinaryCodec) MarshalStable(message interface{}) ([]byte, error) {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return nil, errNotProto(message)
+	}
+	// protobuf does not offer a canonical output today, so this format is not
+	// guaranteed to match deterministic output from other protobuf libraries.
+	// In addition, unknown fields may cause inconsistent output for otherwise
+	// equal messages.
+	// https://github.com/golang/protobuf/issues/1121
+	options := proto.MarshalOptions{Deterministic: true}
+	return options.Marshal(protoMessage)
+}
+
+func (c *protoBinaryCodec) IsBinary() bool {
+	return true
+}
+
+type protoJSONCodec struct {
+	name string
+}
+
+var _ Codec = (*protoJSONCodec)(nil)
+
+func (c *protoJSONCodec) Name() string { return c.name }
+
+func (c *protoJSONCodec) Marshal(message interface{}) ([]byte, error) {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return nil, errNotProto(message)
+	}
+	var options protojson.MarshalOptions
+	return options.Marshal(protoMessage)
+}
+
+func (c *protoJSONCodec) Unmarshal(binary []byte, message interface{}) error {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return errNotProto(message)
+	}
+	if len(binary) == 0 {
+		return errors.New("zero-length payload is not a valid JSON object")
+	}
+	var options protojson.UnmarshalOptions
+	return options.Unmarshal(binary, protoMessage)
+}
+
+func (c *protoJSONCodec) MarshalStable(message interface{}) ([]byte, error) {
+	// protojson does not offer a "deterministic" field ordering, but fields
+	// are still ordered consistently by their index. However, protojson can
+	// output inconsistent whitespace for some reason, therefore it is
+	// suggested to use a formatter to ensure consistent formatting.
+	// https://github.com/golang/protobuf/issues/1373
+	messageJSON, err := c.Marshal(message)
+	if err != nil {
+		return nil, err
+	}
+	compactedJSON := bytes.NewBuffer(messageJSON[:0])
+	if err = json.Compact(compactedJSON, messageJSON); err != nil {
+		return nil, err
+	}
+	return compactedJSON.Bytes(), nil
+}
+
+func (c *protoJSONCodec) IsBinary() bool {
+	return false
+}
+
+// readOnlyCodecs is a read-only interface to a map of named codecs.
+type readOnlyCodecs interface {
+	// Get gets the Codec with the given name.
+	Get(string) Codec
+	// Protobuf gets the user-supplied protobuf codec, falling back to the default
+	// implementation if necessary.
+	//
+	// This is helpful in the gRPC protocol, where the wire protocol requires
+	// marshaling protobuf structs to binary even if the RPC procedures were
+	// generated from a different IDL.
+	Protobuf() Codec
+	// Names returns a copy of the registered codec names. The returned slice is
+	// safe for the caller to mutate.
+	Names() []string
+}
+
+func newReadOnlyCodecs(nameToCodec map[string]Codec) readOnlyCodecs {
+	return &codecMap{
+		nameToCodec: nameToCodec,
+	}
+}
+
+type codecMap struct {
+	nameToCodec map[string]Codec
+}
+
+func (m *codecMap) Get(name string) Codec {
+	return m.nameToCodec[name]
+}
+
+func (m *codecMap) Protobuf() Codec {
+	if pb, ok := m.nameToCodec[codecNameProto]; ok {
+		return pb
+	}
+	return &protoBinaryCodec{}
+}
+
+func (m *codecMap) Names() []string {
+	names := make([]string, 0, len(m.nameToCodec))
+	for name := range m.nameToCodec {
+		names = append(names, name)
+	}
+	return names
+}
+
+func errNotProto(message interface{}) error {
+	if _, ok := message.(protoiface.MessageV1); ok {
+		return fmt.Errorf("%T uses github.com/golang/protobuf, but triple only supports google.golang.org/protobuf: see https://go.dev/blog/protobuf-apiv2", message)
+	}
+	return fmt.Errorf("%T doesn't implement proto.Message", message)
+}
diff --git a/protocol/triple/triple_protocol/codec_test.go b/protocol/triple/triple_protocol/codec_test.go
new file mode 100644
index 0000000..9a2e31f
--- /dev/null
+++ b/protocol/triple/triple_protocol/codec_test.go
@@ -0,0 +1,118 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"strings"
+	"testing"
+	"testing/quick"
+)
+
+import (
+	"google.golang.org/protobuf/proto"
+
+	"google.golang.org/protobuf/types/known/emptypb"
+	"google.golang.org/protobuf/types/known/structpb"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+)
+
+func convertMapToInterface(stringMap map[string]string) map[string]interface{} {
+	interfaceMap := make(map[string]interface{})
+	for key, value := range stringMap {
+		interfaceMap[key] = value
+	}
+	return interfaceMap
+}
+
+func TestCodecRoundTrips(t *testing.T) {
+	t.Parallel()
+	makeRoundtrip := func(codec Codec) func(string, int64) bool {
+		return func(text string, number int64) bool {
+			got := pingv1.PingRequest{}
+			want := pingv1.PingRequest{Text: text, Number: number}
+			data, err := codec.Marshal(&want)
+			if err != nil {
+				t.Fatal(err)
+			}
+			err = codec.Unmarshal(data, &got)
+			if err != nil {
+				t.Fatal(err)
+			}
+			return proto.Equal(&got, &want)
+		}
+	}
+	if err := quick.Check(makeRoundtrip(&protoBinaryCodec{}), nil /* config */); err != nil {
+		t.Error(err)
+	}
+	if err := quick.Check(makeRoundtrip(&protoJSONCodec{}), nil /* config */); err != nil {
+		t.Error(err)
+	}
+}
+
+func TestStableCodec(t *testing.T) {
+	t.Parallel()
+	makeRoundtrip := func(codec stableCodec) func(map[string]string) bool {
+		return func(input map[string]string) bool {
+			initialProto, err := structpb.NewStruct(convertMapToInterface(input))
+			if err != nil {
+				t.Fatal(err)
+			}
+			want, err := codec.MarshalStable(initialProto)
+			if err != nil {
+				t.Fatal(err)
+			}
+			for i := 0; i < 10; i++ {
+				roundtripProto := &structpb.Struct{}
+				err = codec.Unmarshal(want, roundtripProto)
+				if err != nil {
+					t.Fatal(err)
+				}
+				got, err := codec.MarshalStable(roundtripProto)
+				if err != nil {
+					t.Fatal(err)
+				}
+				if !bytes.Equal(got, want) {
+					return false
+				}
+			}
+			return true
+		}
+	}
+	if err := quick.Check(makeRoundtrip(&protoBinaryCodec{}), nil /* config */); err != nil {
+		t.Error(err)
+	}
+	if err := quick.Check(makeRoundtrip(&protoJSONCodec{}), nil /* config */); err != nil {
+		t.Error(err)
+	}
+}
+
+func TestJSONCodec(t *testing.T) {
+	t.Parallel()
+
+	var empty emptypb.Empty
+	codec := &protoJSONCodec{name: "json"}
+	err := codec.Unmarshal([]byte{}, &empty)
+	assert.NotNil(t, err)
+	assert.True(
+		t,
+		strings.Contains(err.Error(), "valid JSON"),
+		assert.Sprintf(`error message should explain that "" is not a valid JSON object`),
+	)
+}
diff --git a/protocol/triple/triple_protocol/compression.go b/protocol/triple/triple_protocol/compression.go
new file mode 100644
index 0000000..d4af026
--- /dev/null
+++ b/protocol/triple/triple_protocol/compression.go
@@ -0,0 +1,215 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"errors"
+	"io"
+	"math"
+	"strings"
+	"sync"
+)
+
+const (
+	compressionGzip     = "gzip"
+	compressionIdentity = "identity"
+)
+
+// A Decompressor is a reusable wrapper that decompresses an underlying data
+// source. The standard library's [*gzip.Reader] implements Decompressor.
+type Decompressor interface {
+	io.Reader
+
+	// Close closes the Decompressor, but not the underlying data source. It may
+	// return an error if the Decompressor wasn't read to EOF.
+	Close() error
+
+	// Reset discards the Decompressor's internal state, if any, and prepares it
+	// to read from a new source of compressed data.
+	Reset(io.Reader) error
+}
+
+// A Compressor is a reusable wrapper that compresses data written to an
+// underlying sink. The standard library's [*gzip.Writer] implements Compressor.
+type Compressor interface {
+	io.Writer
+
+	// Close flushes any buffered data to the underlying sink, then closes the
+	// Compressor. It must not close the underlying sink.
+	Close() error
+
+	// Reset discards the Compressor's internal state, if any, and prepares it to
+	// write compressed data to a new sink.
+	Reset(io.Writer)
+}
+
+type compressionPool struct {
+	decompressors sync.Pool
+	compressors   sync.Pool
+}
+
+func newCompressionPool(
+	newDecompressor func() Decompressor,
+	newCompressor func() Compressor,
+) *compressionPool {
+	if newDecompressor == nil && newCompressor == nil {
+		return nil
+	}
+	return &compressionPool{
+		decompressors: sync.Pool{
+			New: func() interface{} { return newDecompressor() },
+		},
+		compressors: sync.Pool{
+			New: func() interface{} { return newCompressor() },
+		},
+	}
+}
+
+func (c *compressionPool) Decompress(dst *bytes.Buffer, src *bytes.Buffer, readMaxBytes int64) *Error {
+	decompressor, err := c.getDecompressor(src)
+	if err != nil {
+		return errorf(CodeInvalidArgument, "get decompressor: %w", err)
+	}
+	reader := io.Reader(decompressor)
+	if readMaxBytes > 0 && readMaxBytes < math.MaxInt64 {
+		reader = io.LimitReader(decompressor, readMaxBytes+1)
+	}
+	bytesRead, err := dst.ReadFrom(reader)
+	if err != nil {
+		_ = c.putDecompressor(decompressor)
+		return errorf(CodeInvalidArgument, "decompress: %w", err)
+	}
+	if readMaxBytes > 0 && bytesRead > readMaxBytes {
+		discardedBytes, err := io.Copy(io.Discard, decompressor)
+		_ = c.putDecompressor(decompressor)
+		if err != nil {
+			return errorf(CodeResourceExhausted, "message is larger than configured max %d - unable to determine message size: %w", readMaxBytes, err)
+		}
+		return errorf(CodeResourceExhausted, "message size %d is larger than configured max %d", bytesRead+discardedBytes, readMaxBytes)
+	}
+	if err := c.putDecompressor(decompressor); err != nil {
+		return errorf(CodeUnknown, "recycle decompressor: %w", err)
+	}
+	return nil
+}
+
+func (c *compressionPool) Compress(dst *bytes.Buffer, src *bytes.Buffer) *Error {
+	compressor, err := c.getCompressor(dst)
+	if err != nil {
+		return errorf(CodeUnknown, "get compressor: %w", err)
+	}
+	if _, err := io.Copy(compressor, src); err != nil {
+		_ = c.putCompressor(compressor)
+		return errorf(CodeInternal, "compress: %w", err)
+	}
+	if err := c.putCompressor(compressor); err != nil {
+		return errorf(CodeInternal, "recycle compressor: %w", err)
+	}
+	return nil
+}
+
+func (c *compressionPool) getDecompressor(reader io.Reader) (Decompressor, error) {
+	decompressor, ok := c.decompressors.Get().(Decompressor)
+	if !ok {
+		return nil, errors.New("expected Decompressor, got incorrect type from pool")
+	}
+	return decompressor, decompressor.Reset(reader)
+}
+
+func (c *compressionPool) putDecompressor(decompressor Decompressor) error {
+	if err := decompressor.Close(); err != nil {
+		return err
+	}
+	// While it's in the pool, we don't want the decompressor to retain a
+	// reference to the underlying reader. However, most decompressors attempt to
+	// read some header data from the new data source when Reset; since we don't
+	// know the compression format, we can't provide a valid header. Since we
+	// also reset the decompressor when it's pulled out of the pool, we can
+	// ignore errors here.
+	_ = decompressor.Reset(strings.NewReader(""))
+	c.decompressors.Put(decompressor)
+	return nil
+}
+
+func (c *compressionPool) getCompressor(writer io.Writer) (Compressor, error) {
+	compressor, ok := c.compressors.Get().(Compressor)
+	if !ok {
+		return nil, errors.New("expected Compressor, got incorrect type from pool")
+	}
+	compressor.Reset(writer)
+	return compressor, nil
+}
+
+func (c *compressionPool) putCompressor(compressor Compressor) error {
+	if err := compressor.Close(); err != nil {
+		return err
+	}
+	compressor.Reset(io.Discard) // don't keep references
+	c.compressors.Put(compressor)
+	return nil
+}
+
+// readOnlyCompressionPools is a read-only interface to a map of named
+// compressionPools.
+type readOnlyCompressionPools interface {
+	Get(string) *compressionPool
+	Contains(string) bool
+	// Wordy, but clarifies how this is different from readOnlyCodecs.Names().
+	CommaSeparatedNames() string
+}
+
+func newReadOnlyCompressionPools(
+	nameToPool map[string]*compressionPool,
+	reversedNames []string,
+) readOnlyCompressionPools {
+	// Client and handler configs keep compression names in registration order,
+	// but we want the last registered to be the most preferred.
+	names := make([]string, 0, len(reversedNames))
+	seen := make(map[string]struct{}, len(reversedNames))
+	for i := len(reversedNames) - 1; i >= 0; i-- {
+		name := reversedNames[i]
+		if _, ok := seen[name]; ok {
+			continue
+		}
+		seen[name] = struct{}{}
+		names = append(names, name)
+	}
+	return &namedCompressionPools{
+		nameToPool:          nameToPool,
+		commaSeparatedNames: strings.Join(names, ","),
+	}
+}
+
+type namedCompressionPools struct {
+	nameToPool          map[string]*compressionPool
+	commaSeparatedNames string
+}
+
+func (m *namedCompressionPools) Get(name string) *compressionPool {
+	if name == "" || name == compressionIdentity {
+		return nil
+	}
+	return m.nameToPool[name]
+}
+
+func (m *namedCompressionPools) Contains(name string) bool {
+	_, ok := m.nameToPool[name]
+	return ok
+}
+
+func (m *namedCompressionPools) CommaSeparatedNames() string {
+	return m.commaSeparatedNames
+}
diff --git a/protocol/triple/triple_protocol/compression_test.go b/protocol/triple/triple_protocol/compression_test.go
new file mode 100644
index 0000000..0d5da5d
--- /dev/null
+++ b/protocol/triple/triple_protocol/compression_test.go
@@ -0,0 +1,169 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"net/http"
+	"net/http/httptest"
+	"testing"
+)
+
+import (
+	"google.golang.org/protobuf/types/known/emptypb"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestAcceptEncodingOrdering(t *testing.T) {
+	t.Parallel()
+	const (
+		compressionBrotli = "br"
+		expect            = compressionGzip + "," + compressionBrotli
+	)
+
+	withFakeBrotli, ok := withGzip().(*compressionOption)
+	assert.True(t, ok)
+	withFakeBrotli.Name = compressionBrotli
+
+	var called bool
+	verify := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		got := r.Header.Get(tripleUnaryHeaderAcceptCompression)
+		assert.Equal(t, got, expect)
+		w.WriteHeader(http.StatusOK)
+		called = true
+	})
+	server := httptest.NewServer(verify)
+	defer server.Close()
+
+	client := NewClient(
+		server.Client(),
+		server.URL,
+		WithTriple(),
+		withFakeBrotli,
+		withGzip(),
+	)
+	_ = client.CallUnary(context.Background(), NewRequest(&emptypb.Empty{}), NewResponse(&emptypb.Empty{}))
+	assert.True(t, called)
+}
+
+func TestClientCompressionOptionTest(t *testing.T) {
+	t.Parallel()
+	const testURL = "http://foo.bar.com/service/method"
+
+	checkPools := func(t *testing.T, config *clientConfig) {
+		t.Helper()
+		assert.Equal(t, len(config.CompressionNames), len(config.CompressionPools))
+		for _, name := range config.CompressionNames {
+			pool := config.CompressionPools[name]
+			assert.NotNil(t, pool)
+		}
+	}
+	dummyDecompressCtor := func() Decompressor { return nil }
+	dummyCompressCtor := func() Compressor { return nil }
+
+	t.Run("defaults", func(t *testing.T) {
+		t.Parallel()
+		config, err := newClientConfig(testURL, nil)
+		assert.Nil(t, err)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithAcceptCompression", func(t *testing.T) {
+		t.Parallel()
+		opts := []ClientOption{WithAcceptCompression("foo", dummyDecompressCtor, dummyCompressCtor)}
+		config, err := newClientConfig(testURL, opts)
+		assert.Nil(t, err)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip, "foo"})
+		checkPools(t, config)
+	})
+	t.Run("WithAcceptCompression-empty-name-noop", func(t *testing.T) {
+		t.Parallel()
+		opts := []ClientOption{WithAcceptCompression("", dummyDecompressCtor, dummyCompressCtor)}
+		config, err := newClientConfig(testURL, opts)
+		assert.Nil(t, err)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithAcceptCompression-nil-ctors-noop", func(t *testing.T) {
+		t.Parallel()
+		opts := []ClientOption{WithAcceptCompression("foo", nil, nil)}
+		config, err := newClientConfig(testURL, opts)
+		assert.Nil(t, err)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithAcceptCompression-nil-ctors-unregisters", func(t *testing.T) {
+		t.Parallel()
+		opts := []ClientOption{WithAcceptCompression("gzip", nil, nil)}
+		config, err := newClientConfig(testURL, opts)
+		assert.Nil(t, err)
+		assert.Equal(t, config.CompressionNames, []string(nil))
+		checkPools(t, config)
+	})
+}
+
+func TestHandlerCompressionOptionTest(t *testing.T) {
+	t.Parallel()
+	const testProc = "/service/method"
+
+	checkPools := func(t *testing.T, config *handlerConfig) {
+		t.Helper()
+		assert.Equal(t, len(config.CompressionNames), len(config.CompressionPools))
+		for _, name := range config.CompressionNames {
+			pool := config.CompressionPools[name]
+			assert.NotNil(t, pool)
+		}
+	}
+	dummyDecompressCtor := func() Decompressor { return nil }
+	dummyCompressCtor := func() Compressor { return nil }
+
+	t.Run("defaults", func(t *testing.T) {
+		t.Parallel()
+		config := newHandlerConfig(testProc, nil)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithCompression", func(t *testing.T) {
+		t.Parallel()
+		opts := []HandlerOption{WithCompression("foo", dummyDecompressCtor, dummyCompressCtor)}
+		config := newHandlerConfig(testProc, opts)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip, "foo"})
+		checkPools(t, config)
+	})
+	t.Run("WithCompression-empty-name-noop", func(t *testing.T) {
+		t.Parallel()
+		opts := []HandlerOption{WithCompression("", dummyDecompressCtor, dummyCompressCtor)}
+		config := newHandlerConfig(testProc, opts)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithCompression-nil-ctors-noop", func(t *testing.T) {
+		t.Parallel()
+		opts := []HandlerOption{WithCompression("foo", nil, nil)}
+		config := newHandlerConfig(testProc, opts)
+		assert.Equal(t, config.CompressionNames, []string{compressionGzip})
+		checkPools(t, config)
+	})
+	t.Run("WithCompression-nil-ctors-unregisters", func(t *testing.T) {
+		t.Parallel()
+		opts := []HandlerOption{WithCompression("gzip", nil, nil)}
+		config := newHandlerConfig(testProc, opts)
+		assert.Equal(t, config.CompressionNames, []string(nil))
+		checkPools(t, config)
+	})
+}
diff --git a/protocol/triple/triple_protocol/duplex_http_call.go b/protocol/triple/triple_protocol/duplex_http_call.go
new file mode 100644
index 0000000..9f5e6ed
--- /dev/null
+++ b/protocol/triple/triple_protocol/duplex_http_call.go
@@ -0,0 +1,311 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"io"
+	"net/http"
+	"net/url"
+	"sync"
+)
+
+// duplexHTTPCall is a full-duplex stream between the client and server. The
+// request body is the stream from client to server, and the response body is
+// the reverse.
+//
+// Be warned: we need to use some lesser-known APIs to do this with net/http.
+type duplexHTTPCall struct {
+	ctx              context.Context
+	httpClient       HTTPClient
+	streamType       StreamType
+	validateResponse func(*http.Response) *Error
+
+	// We'll use a pipe as the request body. We hand the read side of the pipe to
+	// net/http, and we write to the write side (naturally). The two ends are
+	// safe to use concurrently.
+	requestBodyReader *io.PipeReader
+	requestBodyWriter *io.PipeWriter
+
+	sendRequestOnce sync.Once
+	responseReady   chan struct{}
+	request         *http.Request
+	response        *http.Response
+
+	errMu sync.Mutex
+	err   error
+}
+
+func newDuplexHTTPCall(
+	ctx context.Context,
+	httpClient HTTPClient,
+	url *url.URL,
+	spec Spec,
+	header http.Header,
+) *duplexHTTPCall {
+	// ensure we make a copy of the url before we pass along to the
+	// Request. This ensures if a transport out of our control wants
+	// to mutate the req.URL, we don't feel the effects of it.
+	url = cloneURL(url)
+	pipeReader, pipeWriter := io.Pipe()
+
+	// This is mirroring what http.NewRequestContext did, but
+	// using an already parsed url.URL object, rather than a string
+	// and parsing it again. This is a bit funny with HTTP/1.1
+	// explicitly, but this is logic copied over from
+	// NewRequestContext and doesn't effect the actual version
+	// being transmitted.
+	request := (&http.Request{
+		Method:     http.MethodPost,
+		URL:        url,
+		Header:     header,
+		Proto:      "HTTP/1.1",
+		ProtoMajor: 1,
+		ProtoMinor: 1,
+		Body:       pipeReader,
+		Host:       url.Host,
+	}).WithContext(ctx)
+	return &duplexHTTPCall{
+		ctx:               ctx,
+		httpClient:        httpClient,
+		streamType:        spec.StreamType,
+		requestBodyReader: pipeReader,
+		requestBodyWriter: pipeWriter,
+		request:           request,
+		responseReady:     make(chan struct{}),
+	}
+}
+
+// Write to the request body. Returns an error wrapping io.EOF after SetError
+// is called.
+func (d *duplexHTTPCall) Write(data []byte) (int, error) {
+	// ensure stream has been initialized
+	d.ensureRequestMade()
+	// Before we send any data, check if the context has been canceled.
+	if err := d.ctx.Err(); err != nil {
+		d.SetError(err)
+		return 0, wrapIfContextError(err)
+	}
+	// It's safe to write to this side of the pipe while net/http concurrently
+	// reads from the other side.
+	bytesWritten, err := d.requestBodyWriter.Write(data)
+	if err != nil && errors.Is(err, io.ErrClosedPipe) {
+		// Signal that the stream is closed with the more-typical io.EOF instead of
+		// io.ErrClosedPipe. This makes it easier for protocol-specific wrappers to
+		// match grpc-go's behavior.
+		return bytesWritten, io.EOF
+	}
+	return bytesWritten, err
+}
+
+// Close the request body. Callers *must* call CloseWrite before Read when
+// using HTTP/1.x.
+func (d *duplexHTTPCall) CloseWrite() error {
+	// Even if Write was never called, we need to make an HTTP request. This
+	// ensures that we've sent any headers to the server and that we have an HTTP
+	// response to read from.
+	d.ensureRequestMade()
+	// The user calls CloseWrite to indicate that they're done sending data. It's
+	// safe to close the write side of the pipe while net/http is reading from
+	// it.
+	//
+	// Because triple also supports some RPC types over HTTP/1.1, we need to be
+	// careful how we expose this method to users. HTTP/1.1 doesn't support
+	// bidirectional streaming - the write side of the stream (aka request body)
+	// must be closed before we start reading the response or we'll just block
+	// forever. To make sure users don't have to worry about this, the generated
+	// code for unary, client streaming, and server streaming RPCs must call
+	// CloseWrite automatically rather than requiring the user to do it.
+	return d.requestBodyWriter.Close()
+}
+
+// Header returns the HTTP request headers.
+func (d *duplexHTTPCall) Header() http.Header {
+	return d.request.Header
+}
+
+// Trailer returns the HTTP request trailers.
+func (d *duplexHTTPCall) Trailer() http.Header {
+	return d.request.Trailer
+}
+
+// URL returns the URL for the request.
+func (d *duplexHTTPCall) URL() *url.URL {
+	return d.request.URL
+}
+
+// SetMethod changes the method of the request before it is sent.
+func (d *duplexHTTPCall) SetMethod(method string) {
+	d.request.Method = method
+}
+
+// Read from the response body. Returns the first error passed to SetError.
+func (d *duplexHTTPCall) Read(data []byte) (int, error) {
+	// For sure that server-to-client stream has been initialized
+	// First, we wait until we've gotten the response headers and established the
+	// server-to-client side of the stream.
+	d.BlockUntilResponseReady()
+	if err := d.getError(); err != nil {
+		// The stream is already closed or corrupted.
+		return 0, err
+	}
+	// Before we read, check if the context has been canceled.
+	if err := d.ctx.Err(); err != nil {
+		d.SetError(err)
+		return 0, wrapIfContextError(err)
+	}
+	if d.response == nil {
+		return 0, fmt.Errorf("nil response from %v", d.request.URL)
+	}
+	n, err := d.response.Body.Read(data)
+	return n, wrapIfRSTError(err)
+}
+
+func (d *duplexHTTPCall) CloseRead() error {
+	d.BlockUntilResponseReady()
+	if d.response == nil {
+		return nil
+	}
+	if err := discard(d.response.Body); err != nil {
+		return wrapIfRSTError(err)
+	}
+	return wrapIfRSTError(d.response.Body.Close())
+}
+
+// ResponseStatusCode is the response's HTTP status code.
+func (d *duplexHTTPCall) ResponseStatusCode() (int, error) {
+	d.BlockUntilResponseReady()
+	if d.response == nil {
+		return 0, fmt.Errorf("nil response from %v", d.request.URL)
+	}
+	return d.response.StatusCode, nil
+}
+
+// ResponseHeader returns the response HTTP headers.
+func (d *duplexHTTPCall) ResponseHeader() http.Header {
+	d.BlockUntilResponseReady()
+	if d.response != nil {
+		return d.response.Header
+	}
+	return make(http.Header)
+}
+
+// ResponseTrailer returns the response HTTP trailers.
+func (d *duplexHTTPCall) ResponseTrailer() http.Header {
+	d.BlockUntilResponseReady()
+	if d.response != nil {
+		return d.response.Trailer
+	}
+	return make(http.Header)
+}
+
+// SetError stores any error encountered processing the response. All
+// subsequent calls to Read return this error, and all subsequent calls to
+// Write return an error wrapping io.EOF. It's safe to call concurrently with
+// any other method.
+func (d *duplexHTTPCall) SetError(err error) {
+	d.errMu.Lock()
+	if d.err == nil {
+		d.err = wrapIfContextError(err)
+	}
+	// Closing the read side of the request body pipe acquires an internal lock,
+	// so we want to scope errMu's usage narrowly and avoid defer.
+	d.errMu.Unlock()
+
+	// We've already hit an error, so we should stop writing to the request body.
+	// It's safe to call Close more than once and/or concurrently (calls after
+	// the first are no-ops), so it's okay for us to call this even though
+	// net/http sometimes closes the reader too.
+	//
+	// It's safe to ignore the returned error here. Under the hood, Close calls
+	// CloseWithError, which is documented to always return nil.
+	_ = d.requestBodyReader.Close()
+}
+
+// SetValidateResponse sets the response validation function. The function runs
+// in a background goroutine.
+func (d *duplexHTTPCall) SetValidateResponse(validate func(*http.Response) *Error) {
+	d.validateResponse = validate
+}
+
+func (d *duplexHTTPCall) BlockUntilResponseReady() {
+	<-d.responseReady
+}
+
+func (d *duplexHTTPCall) ensureRequestMade() {
+	d.sendRequestOnce.Do(func() {
+		go d.makeRequest()
+	})
+}
+
+func (d *duplexHTTPCall) makeRequest() {
+	// This runs concurrently with Write and CloseWrite. Read and CloseRead wait
+	// on d.responseReady, so we can't race with them.
+	defer close(d.responseReady)
+
+	// Once we send a message to the server, they send a message back and
+	// establish the receive side of the stream.
+	response, err := d.httpClient.Do(d.request) //nolint:bodyclose
+	if err != nil {
+		// stream knowledge
+		err = wrapIfContextError(err)
+		err = wrapIfLikelyH2CNotConfiguredError(d.request, err)
+		err = wrapIfLikelyWithGRPCNotUsedError(err)
+		err = wrapIfRSTError(err)
+		if _, ok := asError(err); !ok {
+			err = NewError(CodeUnavailable, err)
+		}
+		d.SetError(err)
+		return
+	}
+	d.response = response
+	if err := d.validateResponse(response); err != nil {
+		d.SetError(err)
+		return
+	}
+	if (d.streamType&StreamTypeBidi) == StreamTypeBidi && response.ProtoMajor < 2 {
+		// If we somehow dialed an HTTP/1.x server, fail with an explicit message
+		// rather than returning a more cryptic error later on.
+		d.SetError(errorf(
+			CodeUnimplemented,
+			"response from %v is HTTP/%d.%d: bidi streams require at least HTTP/2",
+			d.request.URL,
+			response.ProtoMajor,
+			response.ProtoMinor,
+		))
+	}
+}
+
+func (d *duplexHTTPCall) getError() error {
+	d.errMu.Lock()
+	defer d.errMu.Unlock()
+	return d.err
+}
+
+// See: https://cs.opensource.google/go/go/+/refs/tags/go1.20.1:src/net/http/clone.go;l=22-33
+func cloneURL(oldURL *url.URL) *url.URL {
+	if oldURL == nil {
+		return nil
+	}
+	newURL := new(url.URL)
+	*newURL = *oldURL
+	if oldURL.User != nil {
+		newURL.User = new(url.Userinfo)
+		*newURL.User = *oldURL.User
+	}
+	return newURL
+}
diff --git a/protocol/triple/triple_protocol/envelope.go b/protocol/triple/triple_protocol/envelope.go
new file mode 100644
index 0000000..845624e
--- /dev/null
+++ b/protocol/triple/triple_protocol/envelope.go
@@ -0,0 +1,279 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"encoding/binary"
+	"errors"
+	"io"
+)
+
+// flagEnvelopeCompressed indicates that the data is compressed. It has the
+// same meaning in the gRPC-Web, gRPC-HTTP2, and Connect protocols.
+const flagEnvelopeCompressed = 0b00000001
+
+var errSpecialEnvelope = errorf(
+	CodeUnknown,
+	"final message has protocol-specific flags: %w",
+	// User code checks for end of stream with errors.Is(err, io.EOF).
+	io.EOF,
+)
+
+// envelope is a block of arbitrary bytes wrapped in gRPC and Connect's framing
+// protocol.
+//
+// Each message is preceded by a 5-byte prefix. The first byte is a uint8 used
+// as a set of bitwise flags, and the remainder is a uint32 indicating the
+// message length. gRPC and Connect interpret the bitwise flags differently, so
+// envelope leaves their interpretation up to the caller.
+type envelope struct {
+	Data  *bytes.Buffer
+	Flags uint8
+}
+
+func (e *envelope) IsSet(flag uint8) bool {
+	return e.Flags&flag == flag
+}
+
+type envelopeWriter struct {
+	writer           io.Writer
+	codec            Codec
+	compressMinBytes int
+	compressionPool  *compressionPool
+	bufferPool       *bufferPool
+	sendMaxBytes     int
+}
+
+// marshal and write to socket
+func (w *envelopeWriter) Marshal(message interface{}) *Error {
+	if message == nil {
+		if _, err := w.writer.Write(nil); err != nil {
+			if connectErr, ok := asError(err); ok {
+				return connectErr
+			}
+			return NewError(CodeUnknown, err)
+		}
+		return nil
+	}
+	raw, err := w.codec.Marshal(message)
+	if err != nil {
+		return errorf(CodeInternal, "marshal message: %w", err)
+	}
+	// We can't avoid allocating the byte slice, so we may as well reuse it once
+	// we're done with it.
+	buffer := bytes.NewBuffer(raw)
+	defer w.bufferPool.Put(buffer)
+	envelope := &envelope{Data: buffer}
+	return w.Write(envelope)
+}
+
+// Write writes the enveloped message, compressing as necessary. It doesn't
+// retain any references to the supplied envelope or its underlying data.
+// so we can reuse it.
+func (w *envelopeWriter) Write(env *envelope) *Error {
+	// compressed || there is no compressionPool || there is no need to compress
+	if env.IsSet(flagEnvelopeCompressed) ||
+		w.compressionPool == nil ||
+		env.Data.Len() < w.compressMinBytes {
+		if w.sendMaxBytes > 0 && env.Data.Len() > w.sendMaxBytes {
+			return errorf(CodeResourceExhausted, "message size %d exceeds sendMaxBytes %d", env.Data.Len(), w.sendMaxBytes)
+		}
+		// write to socket
+		return w.write(env)
+	}
+	data := w.bufferPool.Get()
+	defer w.bufferPool.Put(data)
+	if err := w.compressionPool.Compress(data, env.Data); err != nil {
+		return err
+	}
+	if w.sendMaxBytes > 0 && data.Len() > w.sendMaxBytes {
+		return errorf(CodeResourceExhausted, "compressed message size %d exceeds sendMaxBytes %d", data.Len(), w.sendMaxBytes)
+	}
+	return w.write(&envelope{
+		Data:  data,
+		Flags: env.Flags | flagEnvelopeCompressed,
+	})
+}
+
+func (w *envelopeWriter) write(env *envelope) *Error {
+	prefix := [5]byte{}
+	prefix[0] = env.Flags
+	binary.BigEndian.PutUint32(prefix[1:5], uint32(env.Data.Len()))
+	if _, err := w.writer.Write(prefix[:]); err != nil {
+		if connectErr, ok := asError(err); ok {
+			return connectErr
+		}
+		return errorf(CodeUnknown, "write envelope: %w", err)
+	}
+	if _, err := io.Copy(w.writer, env.Data); err != nil {
+		return errorf(CodeUnknown, "write message: %w", err)
+	}
+	return nil
+}
+
+type envelopeReader struct {
+	reader          io.Reader
+	codec           Codec
+	last            envelope
+	compressionPool *compressionPool
+	bufferPool      *bufferPool
+	readMaxBytes    int
+}
+
+// Unmarshal reads entire envelope and uses codec to unmarshal
+func (r *envelopeReader) Unmarshal(message interface{}) *Error {
+	buffer := r.bufferPool.Get()
+	defer r.bufferPool.Put(buffer)
+
+	env := &envelope{Data: buffer}
+	err := r.Read(env)
+	switch {
+	case err == nil &&
+		(env.Flags == 0 || env.Flags == flagEnvelopeCompressed) &&
+		env.Data.Len() == 0:
+		// This is a standard message (because none of the top 7 bits are set) and
+		// there's no data, so the zero value of the message is correct.
+		return nil
+	case err != nil && errors.Is(err, io.EOF):
+		// The stream has ended. Propagate the EOF to the caller.
+		return err
+	case err != nil:
+		// Something's wrong.
+		return err
+	}
+
+	data := env.Data
+	if data.Len() > 0 && env.IsSet(flagEnvelopeCompressed) {
+		if r.compressionPool == nil {
+			return errorf(
+				CodeInvalidArgument,
+				"gRPC protocol error: sent compressed message without Grpc-Encoding header",
+			)
+		}
+		decompressed := r.bufferPool.Get()
+		defer r.bufferPool.Put(decompressed)
+		if err := r.compressionPool.Decompress(decompressed, data, int64(r.readMaxBytes)); err != nil {
+			return err
+		}
+		data = decompressed
+	}
+
+	if env.Flags != 0 && env.Flags != flagEnvelopeCompressed {
+		// One of the protocol-specific flags are set, so this is the end of the
+		// stream. Save the message for protocol-specific code to process and
+		// return a sentinel error. Since we've deferred functions to return env's
+		// underlying buffer to a pool, we need to keep a copy.
+		r.last = envelope{
+			Data:  r.bufferPool.Get(),
+			Flags: env.Flags,
+		}
+		// Don't return last to the pool! We're going to reference the data
+		// elsewhere.
+		if _, err := r.last.Data.ReadFrom(data); err != nil {
+			return errorf(CodeUnknown, "copy final envelope: %w", err)
+		}
+		return errSpecialEnvelope
+	}
+
+	if err := r.codec.Unmarshal(data.Bytes(), message); err != nil {
+		return errorf(CodeInvalidArgument, "unmarshal into %T: %w", message, err)
+	}
+	return nil
+}
+
+func (r *envelopeReader) Read(env *envelope) *Error {
+	// Read prefix firstly, then read the packet with length specified by length field
+	prefixes := [5]byte{}
+	prefixBytesRead, err := r.reader.Read(prefixes[:])
+
+	switch {
+	case (err == nil || errors.Is(err, io.EOF)) &&
+		prefixBytesRead == 5 &&
+		isSizeZeroPrefix(prefixes):
+		// Successfully read prefix and expect no additional data.
+		env.Flags = prefixes[0]
+		return nil
+	case err != nil && errors.Is(err, io.EOF) && prefixBytesRead == 0:
+		// The stream ended cleanly. That's expected, but we need to propagate them
+		// to the user so that they know that the stream has ended. We shouldn't
+		// add any alarming text about protocol errors, though.
+		return NewError(CodeUnknown, err)
+	case err != nil || prefixBytesRead < 5:
+		// Something else has gone wrong - the stream didn't end cleanly.
+		if connectErr, ok := asError(err); ok {
+			return connectErr
+		}
+		if maxBytesErr := asMaxBytesError(err, "read 5 byte message prefix"); maxBytesErr != nil {
+			// We're reading from an http.MaxBytesHandler, and we've exceeded the read limit.
+			return maxBytesErr
+		}
+		return errorf(
+			CodeInvalidArgument,
+			"protocol error: incomplete envelope: %w", err,
+		)
+	}
+	size := int(binary.BigEndian.Uint32(prefixes[1:5]))
+	if size < 0 {
+		return errorf(CodeInvalidArgument, "message size %d overflowed uint32", size)
+	}
+	if r.readMaxBytes > 0 && size > r.readMaxBytes {
+		_, err := io.CopyN(io.Discard, r.reader, int64(size))
+		if err != nil && !errors.Is(err, io.EOF) {
+			return errorf(CodeUnknown, "read enveloped message: %w", err)
+		}
+		return errorf(CodeResourceExhausted, "message size %d is larger than configured max %d", size, r.readMaxBytes)
+	}
+	if size > 0 {
+		env.Data.Grow(size)
+		// At layer 7, we don't know exactly what's happening down in L4. Large
+		// length-prefixed messages may arrive in chunks, so we may need to read
+		// the request body past EOF. We also need to take care that we don't retry
+		// forever if the message is malformed.
+		remaining := int64(size)
+		for remaining > 0 {
+			bytesRead, err := io.CopyN(env.Data, r.reader, remaining)
+			if err != nil && !errors.Is(err, io.EOF) {
+				if maxBytesErr := asMaxBytesError(err, "read %d byte message", size); maxBytesErr != nil {
+					// We're reading from an http.MaxBytesHandler, and we've exceeded the read limit.
+					return maxBytesErr
+				}
+				return errorf(CodeUnknown, "read enveloped message: %w", err)
+			}
+			if errors.Is(err, io.EOF) && bytesRead == 0 {
+				// We've gotten zero-length chunk of data. Message is likely malformed,
+				// don't wait for additional chunks.
+				return errorf(
+					CodeInvalidArgument,
+					"protocol error: promised %d bytes in enveloped message, got %d bytes",
+					size,
+					int64(size)-remaining,
+				)
+			}
+			remaining -= bytesRead
+		}
+	}
+	env.Flags = prefixes[0]
+	return nil
+}
+
+func isSizeZeroPrefix(prefix [5]byte) bool {
+	for i := 1; i < 5; i++ {
+		if prefix[i] != 0 {
+			return false
+		}
+	}
+	return true
+}
diff --git a/protocol/triple/triple_protocol/error.go b/protocol/triple/triple_protocol/error.go
new file mode 100644
index 0000000..a277cfb
--- /dev/null
+++ b/protocol/triple/triple_protocol/error.go
@@ -0,0 +1,371 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"net/http"
+	"net/url"
+	"strings"
+)
+
+import (
+	"google.golang.org/protobuf/proto"
+
+	"google.golang.org/protobuf/types/known/anypb"
+)
+
+const (
+	commonErrorsURL          = "https://connect.build/docs/go/common-errors"
+	defaultAnyResolverPrefix = "type.googleapis.com/"
+)
+
+// An ErrorDetail is a self-describing Protobuf message attached to an [*Error].
+// Error details are sent over the network to clients, which can then work with
+// strongly-typed data rather than trying to parse a complex error message. For
+// example, you might use details to send a localized error message or retry
+// parameters to the client.
+//
+// The [google.golang.org/genproto/googleapis/rpc/errdetails] package contains a
+// variety of Protobuf messages commonly used as error details.
+type ErrorDetail struct {
+	pb       *anypb.Any
+	wireJSON string // preserve human-readable JSON
+}
+
+// NewErrorDetail constructs a new error detail. If msg is an *[anypb.Any] then
+// it is used as is. Otherwise, it is first marshalled into an *[anypb.Any]
+// value. This returns an error if msg cannot be marshalled.
+func NewErrorDetail(msg proto.Message) (*ErrorDetail, error) {
+	// If it's already an Any, don't wrap it inside another.
+	if pb, ok := msg.(*anypb.Any); ok {
+		return &ErrorDetail{pb: pb}, nil
+	}
+	pb, err := anypb.New(msg)
+	if err != nil {
+		return nil, err
+	}
+	return &ErrorDetail{pb: pb}, nil
+}
+
+// Type is the fully-qualified name of the detail's Protobuf message (for
+// example, acme.foo.v1.FooDetail).
+func (d *ErrorDetail) Type() string {
+	// proto.Any tries to make messages self-describing by using type URLs rather
+	// than plain type names, but there aren't any descriptor registries
+	// deployed. With the current state of the `Any` code, it's not possible to
+	// build a useful type registry either. To hide this from users, we should
+	// trim the static hostname that `Any` adds to the type name.
+	//
+	// If we ever want to support remote registries, we can add an explicit
+	// `TypeURL` method.
+	return strings.TrimPrefix(d.pb.TypeUrl, defaultAnyResolverPrefix)
+}
+
+// Bytes returns a copy of the Protobuf-serialized detail.
+func (d *ErrorDetail) Bytes() []byte {
+	out := make([]byte, len(d.pb.Value))
+	copy(out, d.pb.Value)
+	return out
+}
+
+// Value uses the Protobuf runtime's package-global registry to unmarshal the
+// Detail into a strongly-typed message. Typically, clients use Go type
+// assertions to cast from the proto.Message interface to concrete types.
+func (d *ErrorDetail) Value() (proto.Message, error) {
+	return d.pb.UnmarshalNew()
+}
+
+// An Error captures four key pieces of information: a [Code], an underlying Go
+// error, a map of metadata, and an optional collection of arbitrary Protobuf
+// messages called "details" (more on those below). Servers send the code, the
+// underlying error's Error() output, the metadata, and details over the wire
+// to clients. Remember that the underlying error's message will be sent to
+// clients - take care not to leak sensitive information from public APIs!
+//
+// Service implementations and interceptors should return errors that can be
+// cast to an [*Error] (using the standard library's [errors.As]). If the returned
+// error can't be cast to an [*Error], triple will use [CodeUnknown] and the
+// returned error's message.
+//
+// Error details are an optional mechanism for servers, interceptors, and
+// proxies to attach arbitrary Protobuf messages to the error code and message.
+// They're a clearer and more performant alternative to HTTP header
+// microformats. See [the documentation on errors] for more details.
+//
+// [the documentation on errors]: https://connect.build/docs/go/errors
+type Error struct {
+	code    Code
+	err     error
+	details []*ErrorDetail
+	meta    http.Header
+	wireErr bool
+}
+
+// NewError annotates any Go error with a status code.
+func NewError(c Code, underlying error) *Error {
+	return &Error{code: c, err: underlying}
+}
+
+// NewWireError is similar to [NewError], but the resulting *Error returns true
+// when tested with [IsWireError].
+//
+// This is useful for clients trying to propagate partial failures from
+// streaming RPCs. Often, these RPCs include error information in their
+// response messages (for example, [gRPC server reflection] and
+// OpenTelemtetry's [OTLP]). Clients propagating these errors up the stack
+// should use NewWireError to clarify that the error code, message, and details
+// (if any) were explicitly sent by the server rather than inferred from a
+// lower-level networking error or timeout.
+//
+// [gRPC server reflection]: https://github.com/grpc/grpc/blob/v1.49.2/src/proto/grpc/reflection/v1alpha/reflection.proto#L132-L136
+// [OTLP]: https://github.com/open-telemetry/opentelemetry-specification/blob/main/specification/protocol/otlp.md#partial-success
+func NewWireError(c Code, underlying error) *Error {
+	err := NewError(c, underlying)
+	err.wireErr = true
+	return err
+}
+
+// IsWireError checks whether the error was returned by the server, as opposed
+// to being synthesized by the client.
+//
+// Clients may find this useful when deciding how to propagate errors. For
+// example, an RPC-to-HTTP proxy might expose a server-sent CodeUnknown as an
+// HTTP 500 but a client-synthesized CodeUnknown as a 503.
+func IsWireError(err error) bool {
+	se := new(Error)
+	if !errors.As(err, &se) {
+		return false
+	}
+	return se.wireErr
+}
+
+func (e *Error) Error() string {
+	message := e.Message()
+	if message == "" {
+		return e.code.String()
+	}
+	return e.code.String() + ": " + message
+}
+
+// Message returns the underlying error message. It may be empty if the
+// original error was created with a status code and a nil error.
+func (e *Error) Message() string {
+	if e.err != nil {
+		return e.err.Error()
+	}
+	return ""
+}
+
+// Unwrap allows [errors.Is] and [errors.As] access to the underlying error.
+func (e *Error) Unwrap() error {
+	return e.err
+}
+
+// Code returns the error's status code.
+func (e *Error) Code() Code {
+	return e.code
+}
+
+// Details returns the error's details.
+func (e *Error) Details() []*ErrorDetail {
+	return e.details
+}
+
+// AddDetail appends to the error's details.
+func (e *Error) AddDetail(d *ErrorDetail) {
+	e.details = append(e.details, d)
+}
+
+// Meta allows the error to carry additional information as key-value pairs.
+//
+// Metadata attached to errors returned by unary handlers is always sent as
+// HTTP headers, regardless of the protocol. Metadata attached to errors
+// returned by streaming handlers may be sent as HTTP headers, HTTP trailers,
+// or a block of in-body metadata, depending on the protocol in use and whether
+// or not the handler has already written messages to the stream.
+//
+// When clients receive errors, the metadata contains the union of the HTTP
+// headers and the protocol-specific trailers (either HTTP trailers or in-body
+// metadata).
+func (e *Error) Meta() http.Header {
+	if e.meta == nil {
+		e.meta = make(http.Header)
+	}
+	return e.meta
+}
+
+func (e *Error) detailsAsAny() []*anypb.Any {
+	anys := make([]*anypb.Any, 0, len(e.details))
+	for _, detail := range e.details {
+		anys = append(anys, detail.pb)
+	}
+	return anys
+}
+
+// errorf calls fmt.Errorf with the supplied template and arguments, then wraps
+// the resulting error.
+func errorf(c Code, template string, args ...interface{}) *Error {
+	return NewError(c, fmt.Errorf(template, args...))
+}
+
+// asError uses errors.As to unwrap any error and look for a triple *Error.
+func asError(err error) (*Error, bool) {
+	var connectErr *Error
+	ok := errors.As(err, &connectErr)
+	return connectErr, ok
+}
+
+// wrapIfUncoded ensures that all errors are wrapped. It leaves already-wrapped
+// errors unchanged, uses wrapIfContextError to apply codes to context.Canceled
+// and context.DeadlineExceeded, and falls back to wrapping other errors with
+// CodeUnknown.
+func wrapIfUncoded(err error) error {
+	if err == nil {
+		return nil
+	}
+	maybeCodedErr := wrapIfContextError(err)
+	if _, ok := asError(maybeCodedErr); ok {
+		return maybeCodedErr
+	}
+	return NewError(CodeUnknown, maybeCodedErr)
+}
+
+// wrapIfContextError applies CodeCanceled or CodeDeadlineExceeded to Go's
+// context.Canceled and context.DeadlineExceeded errors, but only if they
+// haven't already been wrapped.
+func wrapIfContextError(err error) error {
+	if err == nil {
+		return nil
+	}
+	if _, ok := asError(err); ok {
+		return err
+	}
+	if errors.Is(err, context.Canceled) {
+		return NewError(CodeCanceled, err)
+	}
+	if errors.Is(err, context.DeadlineExceeded) {
+		return NewError(CodeDeadlineExceeded, err)
+	}
+	return err
+}
+
+// wrapIfLikelyWithGRPCNotUsedError adds a wrapping error that has a message
+// telling the caller that they likely need to use h2c but are using a raw http.Client{}.
+//
+// This happens when running a gRPC-only server.
+// This is fragile and may break over time, and this should be considered a best-effort.
+func wrapIfLikelyH2CNotConfiguredError(request *http.Request, err error) error {
+	if err == nil {
+		return nil
+	}
+	if _, ok := asError(err); ok {
+		return err
+	}
+	if url := request.URL; url != nil && url.Scheme != "http" {
+		// If the scheme is not http, we definitely do not have an h2c error, so just return.
+		return err
+	}
+	// net/http code has been investigated and there is no typing of any of these errors
+	// they are all created with fmt.Errorf
+	// grpc-go returns the first error 2/3-3/4 of the time, and the second error 1/4-1/3 of the time
+	if errString := err.Error(); strings.HasPrefix(errString, `Post "`) &&
+		(strings.Contains(errString, `net/http: HTTP/1.x transport connection broken: malformed HTTP response`) ||
+			strings.HasSuffix(errString, `write: broken pipe`)) {
+		return fmt.Errorf("possible h2c configuration issue when talking to gRPC server, see %s: %w", commonErrorsURL, err)
+	}
+	return err
+}
+
+// wrapIfLikelyWithGRPCNotUsedError adds a wrapping error that has a message
+// telling the caller that they likely forgot to use triple.WithGRPC().
+//
+// This happens when running a gRPC-only server.
+// This is fragile and may break over time, and this should be considered a best-effort.
+func wrapIfLikelyWithGRPCNotUsedError(err error) error {
+	if err == nil {
+		return nil
+	}
+	if _, ok := asError(err); ok {
+		return err
+	}
+	// golang.org/x/net code has been investigated and there is no typing of this error
+	// it is created with fmt.Errorf
+	// http2/transport.go:573:	return nil, fmt.Errorf("http2: Transport: cannot retry err [%v] after Request.Body was written; define Request.GetBody to avoid this error", err)
+	if errString := err.Error(); strings.HasPrefix(errString, `Post "`) &&
+		strings.Contains(errString, `http2: Transport: cannot retry err`) &&
+		strings.HasSuffix(errString, `after Request.Body was written; define Request.GetBody to avoid this error`) {
+		return fmt.Errorf("possible missing triple.WithGPRC() client option when talking to gRPC server, see %s: %w", commonErrorsURL, err)
+	}
+	return err
+}
+
+// HTTP/2 has its own set of error codes, which it sends in RST_STREAM frames.
+// When the server sends one of these errors, we should map it back into our
+// RPC error codes following
+// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#http2-transport-mapping.
+//
+// This would be vastly simpler if we were using x/net/http2 directly, since
+// the StreamError type is exported. When x/net/http2 gets vendored into
+// net/http, though, all these types become unexported...so we're left with
+// string munging.
+func wrapIfRSTError(err error) error {
+	const (
+		streamErrPrefix = "stream error: "
+		fromPeerSuffix  = "; received from peer"
+	)
+	if err == nil {
+		return nil
+	}
+	if _, ok := asError(err); ok {
+		return err
+	}
+	if urlErr := new(url.Error); errors.As(err, &urlErr) {
+		// If we get an RST_STREAM error from http.Client.Do, it's wrapped in a
+		// *url.Error.
+		err = urlErr.Unwrap()
+	}
+	msg := err.Error()
+	if !strings.HasPrefix(msg, streamErrPrefix) {
+		return err
+	}
+	if !strings.HasSuffix(msg, fromPeerSuffix) {
+		return err
+	}
+	msg = strings.TrimSuffix(msg, fromPeerSuffix)
+	i := strings.LastIndex(msg, ";")
+	if i < 0 || i >= len(msg)-1 {
+		return err
+	}
+	msg = msg[i+1:]
+	msg = strings.TrimSpace(msg)
+	switch msg {
+	case "NO_ERROR", "PROTOCOL_ERROR", "INTERNAL_ERROR", "FLOW_CONTROL_ERROR",
+		"SETTINGS_TIMEOUT", "FRAME_SIZE_ERROR", "COMPRESSION_ERROR", "CONNECT_ERROR":
+		return NewError(CodeInternal, err)
+	case "REFUSED_STREAM":
+		return NewError(CodeUnavailable, err)
+	case "CANCEL":
+		return NewError(CodeCanceled, err)
+	case "ENHANCE_YOUR_CALM":
+		return NewError(CodeResourceExhausted, fmt.Errorf("bandwidth exhausted: %w", err))
+	case "INADEQUATE_SECURITY":
+		return NewError(CodePermissionDenied, fmt.Errorf("transport protocol insecure: %w", err))
+	default:
+		return err
+	}
+}
diff --git a/protocol/triple/triple_protocol/error_example_test.go b/protocol/triple/triple_protocol/error_example_test.go
new file mode 100644
index 0000000..392c8f8
--- /dev/null
+++ b/protocol/triple/triple_protocol/error_example_test.go
@@ -0,0 +1,37 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"errors"
+	"fmt"
+)
+
+import (
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+)
+
+func ExampleError_Message() {
+	err := fmt.Errorf(
+		"another: %w",
+		tri.NewError(tri.CodeUnavailable, errors.New("failed to foo")),
+	)
+	if connectErr := (&tri.Error{}); errors.As(err, &connectErr) {
+		fmt.Println("underlying error message:", connectErr.Message())
+	}
+
+	// Output:
+	// underlying error message: failed to foo
+}
diff --git a/protocol/triple/triple_protocol/error_test.go b/protocol/triple/triple_protocol/error_test.go
new file mode 100644
index 0000000..6187879
--- /dev/null
+++ b/protocol/triple/triple_protocol/error_test.go
@@ -0,0 +1,115 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"errors"
+	"fmt"
+	"strings"
+	"testing"
+	"time"
+)
+
+import (
+	"google.golang.org/protobuf/proto"
+
+	"google.golang.org/protobuf/types/known/durationpb"
+	"google.golang.org/protobuf/types/known/emptypb"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestErrorNilUnderlying(t *testing.T) {
+	t.Parallel()
+	err := NewError(CodeUnknown, nil)
+	assert.NotNil(t, err)
+	assert.Equal(t, err.Error(), CodeUnknown.String())
+	assert.Equal(t, err.Code(), CodeUnknown)
+	assert.Zero(t, err.Details())
+	detail, detailErr := NewErrorDetail(&emptypb.Empty{})
+	assert.Nil(t, detailErr)
+	err.AddDetail(detail)
+	assert.Equal(t, len(err.Details()), 1)
+	assert.Equal(t, err.Details()[0].Type(), "google.protobuf.Empty")
+	err.Meta().Set("foo", "bar")
+	assert.Equal(t, err.Meta().Get("foo"), "bar")
+	assert.Equal(t, CodeOf(err), CodeUnknown)
+}
+
+func TestErrorFormatting(t *testing.T) {
+	t.Parallel()
+	assert.Equal(
+		t,
+		NewError(CodeUnavailable, errors.New("")).Error(),
+		CodeUnavailable.String(),
+	)
+	got := NewError(CodeUnavailable, errors.New("foo")).Error()
+	assert.True(t, strings.Contains(got, CodeUnavailable.String()))
+	assert.True(t, strings.Contains(got, "foo"))
+}
+
+func TestErrorCode(t *testing.T) {
+	t.Parallel()
+	err := fmt.Errorf(
+		"another: %w",
+		NewError(CodeUnavailable, errors.New("foo")),
+	)
+	tripleErr, ok := asError(err)
+	assert.True(t, ok)
+	assert.Equal(t, tripleErr.Code(), CodeUnavailable)
+}
+
+func TestCodeOf(t *testing.T) {
+	t.Parallel()
+	assert.Equal(
+		t,
+		CodeOf(NewError(CodeUnavailable, errors.New("foo"))),
+		CodeUnavailable,
+	)
+	assert.Equal(t, CodeOf(errors.New("foo")), CodeUnknown)
+}
+
+func TestErrorDetails(t *testing.T) {
+	t.Parallel()
+	second := durationpb.New(time.Second)
+	detail, err := NewErrorDetail(second)
+	assert.Nil(t, err)
+	tripleErr := NewError(CodeUnknown, errors.New("error with details"))
+	assert.Zero(t, tripleErr.Details())
+	tripleErr.AddDetail(detail)
+	assert.Equal(t, len(tripleErr.Details()), 1)
+	unmarshaled, err := tripleErr.Details()[0].Value()
+	assert.Nil(t, err)
+	assert.Equal(t, unmarshaled, proto.Message(second))
+	secondBin, err := proto.Marshal(second)
+	assert.Nil(t, err)
+	assert.Equal(t, detail.Bytes(), secondBin)
+}
+
+func TestErrorIs(t *testing.T) {
+	t.Parallel()
+	// errors.New and fmt.Errorf return *errors.errorString. errors.Is
+	// considers two *errors.errorStrings equal iff they have the same address.
+	err := errors.New("oh no")
+	assert.False(t, errors.Is(err, errors.New("oh no")))
+	assert.True(t, errors.Is(err, err))
+	// Our errors should have the same semantics. Note that we'd need to extend
+	// the ErrorDetail interface to support value equality.
+	tripleErr := NewError(CodeUnavailable, err)
+	assert.False(t, errors.Is(tripleErr, NewError(CodeUnavailable, err)))
+	assert.True(t, errors.Is(tripleErr, tripleErr))
+}
diff --git a/protocol/triple/triple_protocol/error_writer.go b/protocol/triple/triple_protocol/error_writer.go
new file mode 100644
index 0000000..b9aedef
--- /dev/null
+++ b/protocol/triple/triple_protocol/error_writer.go
@@ -0,0 +1,159 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"encoding/json"
+	"fmt"
+	"net/http"
+	"strings"
+)
+
+// An ErrorWriter writes errors to an [http.ResponseWriter] in the format
+// expected by an RPC client. This is especially useful in server-side net/http
+// middleware, where you may wish to handle requests from RPC and non-RPC
+// clients with the same code.
+//
+// ErrorWriters are safe to use concurrently.
+type ErrorWriter struct {
+	bufferPool                   *bufferPool
+	protobuf                     Codec
+	allContentTypes              map[string]struct{}
+	grpcContentTypes             map[string]struct{}
+	grpcWebContentTypes          map[string]struct{}
+	unaryConnectContentTypes     map[string]struct{}
+	streamingConnectContentTypes map[string]struct{}
+}
+
+// NewErrorWriter constructs an ErrorWriter. To properly recognize supported
+// RPC Content-Types in net/http middleware, you must pass the same
+// HandlerOptions to NewErrorWriter and any wrapped Connect handlers.
+func NewErrorWriter(opts ...HandlerOption) *ErrorWriter {
+	config := newHandlerConfig("", opts)
+	writer := &ErrorWriter{
+		bufferPool:                   config.BufferPool,
+		protobuf:                     newReadOnlyCodecs(config.Codecs).Protobuf(),
+		allContentTypes:              make(map[string]struct{}),
+		grpcContentTypes:             make(map[string]struct{}),
+		grpcWebContentTypes:          make(map[string]struct{}),
+		unaryConnectContentTypes:     make(map[string]struct{}),
+		streamingConnectContentTypes: make(map[string]struct{}),
+	}
+	for name := range config.Codecs {
+		unary := tripleContentTypeFromCodecName(StreamTypeUnary, name)
+		writer.allContentTypes[unary] = struct{}{}
+		writer.unaryConnectContentTypes[unary] = struct{}{}
+		streaming := tripleContentTypeFromCodecName(StreamTypeBidi, name)
+		writer.streamingConnectContentTypes[streaming] = struct{}{}
+		writer.allContentTypes[streaming] = struct{}{}
+	}
+	if config.HandleGRPC {
+		writer.grpcContentTypes[grpcContentTypeDefault] = struct{}{}
+		writer.allContentTypes[grpcContentTypeDefault] = struct{}{}
+		for name := range config.Codecs {
+			ct := grpcContentTypeFromCodecName(name)
+			writer.grpcContentTypes[ct] = struct{}{}
+			writer.allContentTypes[ct] = struct{}{}
+		}
+	}
+	return writer
+}
+
+// IsSupported checks whether a request is using one of the ErrorWriter's
+// supported RPC protocols.
+func (w *ErrorWriter) IsSupported(request *http.Request) bool {
+	ctype := canonicalizeContentType(getHeaderCanonical(request.Header, headerContentType))
+	_, ok := w.allContentTypes[ctype]
+	return ok
+}
+
+// Write an error, using the format appropriate for the RPC protocol in use.
+// Callers should first use IsSupported to verify that the request is using one
+// of the ErrorWriter's supported RPC protocols.
+//
+// Write does not read or close the request body.
+func (w *ErrorWriter) Write(response http.ResponseWriter, request *http.Request, err error) error {
+	ctype := canonicalizeContentType(getHeaderCanonical(request.Header, headerContentType))
+	if _, ok := w.unaryConnectContentTypes[ctype]; ok {
+		// Unary errors are always JSON.
+		setHeaderCanonical(response.Header(), headerContentType, tripleUnaryContentTypeJSON)
+		return w.writeConnectUnary(response, err)
+	}
+	//if _, ok := w.streamingConnectContentTypes[ctype]; ok {
+	//	setHeaderCanonical(response.Header(), headerContentType, ctype)
+	//	return w.writeConnectStreaming(response, err)
+	//}
+	if _, ok := w.grpcContentTypes[ctype]; ok {
+		setHeaderCanonical(response.Header(), headerContentType, ctype)
+		return w.writeGRPC(response, err)
+	}
+	if _, ok := w.grpcWebContentTypes[ctype]; ok {
+		setHeaderCanonical(response.Header(), headerContentType, ctype)
+		return w.writeGRPCWeb(response, err)
+	}
+	return fmt.Errorf("unsupported Content-Type %q", ctype)
+}
+
+func (w *ErrorWriter) writeConnectUnary(response http.ResponseWriter, err error) error {
+	if tripleErr, ok := asError(err); ok {
+		mergeHeaders(response.Header(), tripleErr.meta)
+	}
+	response.WriteHeader(tripleCodeToHTTP(CodeOf(err)))
+	data, marshalErr := json.Marshal(newTripleWireError(err))
+	if marshalErr != nil {
+		return fmt.Errorf("marshal error: %w", marshalErr)
+	}
+	_, writeErr := response.Write(data)
+	return writeErr
+}
+
+//func (w *ErrorWriter) writeConnectStreaming(response http.ResponseWriter, err error) error {
+//	response.WriteHeader(http.StatusOK)
+//	marshaler := &connectStreamingMarshaler{
+//		envelopeWriter: envelopeWriter{
+//			writer:     response,
+//			bufferPool: w.bufferPool,
+//		},
+//	}
+//	// MarshalEndStream returns *Error: check return value to avoid typed nils.
+//	if marshalErr := marshaler.MarshalEndStream(err, make(http.Header)); marshalErr != nil {
+//		return marshalErr
+//	}
+//	return nil
+//}
+
+func (w *ErrorWriter) writeGRPC(response http.ResponseWriter, err error) error {
+	trailers := make(http.Header, 2) // need space for at least code & message
+	grpcErrorToTrailer(w.bufferPool, trailers, w.protobuf, err)
+	// To make net/http reliably send trailers without a body, we must set the
+	// Trailers header rather than using http.TrailerPrefix. See
+	// https://github.com/golang/go/issues/54723.
+	keys := make([]string, 0, len(trailers))
+	for k := range trailers {
+		keys = append(keys, k)
+	}
+	setHeaderCanonical(response.Header(), headerTrailer, strings.Join(keys, ","))
+	response.WriteHeader(http.StatusOK)
+	mergeHeaders(response.Header(), trailers)
+	return nil
+}
+
+func (w *ErrorWriter) writeGRPCWeb(response http.ResponseWriter, err error) error {
+	// This is a trailers-only response. To match the behavior of Envoy and
+	// protocol_grpc.go, put the trailers in the HTTP headers.
+	grpcErrorToTrailer(w.bufferPool, response.Header(), w.protobuf, err)
+	response.WriteHeader(http.StatusOK)
+	return nil
+}
diff --git a/protocol/triple/triple_protocol/error_writer_example_test.go b/protocol/triple/triple_protocol/error_writer_example_test.go
new file mode 100644
index 0000000..ae51f41
--- /dev/null
+++ b/protocol/triple/triple_protocol/error_writer_example_test.go
@@ -0,0 +1,71 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"errors"
+	"io"
+	"log"
+	"net/http"
+)
+
+import (
+	tri "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+)
+
+// NewHelloHandler is an example HTTP handler. In a real application, it might
+// handle RPCs, requests for HTML, or anything else.
+func NewHelloHandler() http.Handler {
+	return http.HandlerFunc(func(response http.ResponseWriter, request *http.Request) {
+		io.WriteString(response, "Hello, world!")
+	})
+}
+
+// NewAuthenticatedHandler is an example of middleware that works with both RPC
+// and non-RPC clients.
+func NewAuthenticatedHandler(handler http.Handler) http.Handler {
+	errorWriter := tri.NewErrorWriter()
+	return http.HandlerFunc(func(response http.ResponseWriter, request *http.Request) {
+		// Dummy authentication logic.
+		if request.Header.Get("Token") == "super-secret" {
+			handler.ServeHTTP(response, request)
+			return
+		}
+		defer request.Body.Close()
+		defer io.Copy(io.Discard, request.Body)
+		if errorWriter.IsSupported(request) {
+			// Send a protocol-appropriate error to RPC clients, so that they receive
+			// the right code, message, and any metadata or error details.
+			unauthenticated := tri.NewError(tri.CodeUnauthenticated, errors.New("invalid token"))
+			errorWriter.Write(response, request, unauthenticated)
+		} else {
+			// Send an error to non-RPC clients.
+			response.WriteHeader(http.StatusUnauthorized)
+			io.WriteString(response, "invalid token")
+		}
+	})
+}
+
+func ExampleErrorWriter() {
+	mux := http.NewServeMux()
+	mux.Handle("/", NewHelloHandler())
+	srv := &http.Server{
+		Addr:    ":8080",
+		Handler: NewAuthenticatedHandler(mux),
+	}
+	if err := srv.ListenAndServe(); err != nil && !errors.Is(err, http.ErrServerClosed) {
+		log.Fatalln(err)
+	}
+}
diff --git a/protocol/triple/triple_protocol/example_init_test.go b/protocol/triple/triple_protocol/example_init_test.go
new file mode 100644
index 0000000..6d28df7
--- /dev/null
+++ b/protocol/triple/triple_protocol/example_init_test.go
@@ -0,0 +1,146 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"errors"
+	"net"
+	"net/http"
+	"net/http/httptest"
+	"sync"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+var examplePingServer *inMemoryServer
+
+func init() {
+	// Generally, init functions are bad.
+	//
+	// To write testable examples that users can grok *and* can execute in the
+	// playground, where networking is disabled, we need an HTTP server that uses
+	// in-memory pipes instead of TCP. We don't want to pollute every example
+	// with this setup code.
+	//
+	// The least-awful option is to set up the server in init().
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	examplePingServer = newInMemoryServer(mux)
+}
+
+// inMemoryServer is an HTTP server that uses in-memory pipes instead of TCP.
+// It supports HTTP/2 and has TLS enabled.
+//
+// The Go Playground panics if we try to start a TCP-backed server. If you're
+// not familiar with the Playground's behavior, it looks like our examples are
+// broken. This server lets us write examples that work in the playground
+// without abstracting over HTTP.
+type inMemoryServer struct {
+	server   *httptest.Server
+	listener *memoryListener
+}
+
+// newInMemoryServer constructs and starts an inMemoryServer.
+func newInMemoryServer(handler http.Handler) *inMemoryServer {
+	lis := &memoryListener{
+		conns:  make(chan net.Conn),
+		closed: make(chan struct{}),
+	}
+	server := httptest.NewUnstartedServer(handler)
+	server.Listener = lis
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	return &inMemoryServer{
+		server:   server,
+		listener: lis,
+	}
+}
+
+// Client returns an HTTP client configured to trust the server's TLS
+// certificate and use HTTP/2 over an in-memory pipe. Automatic HTTP-level gzip
+// compression is disabled. It closes its idle connections when the server is
+// closed.
+func (s *inMemoryServer) Client() *http.Client {
+	client := s.server.Client()
+	if transport, ok := client.Transport.(*http.Transport); ok {
+		transport.DialContext = s.listener.DialContext
+		transport.DisableCompression = true
+	}
+	return client
+}
+
+// URL is the server's URL.
+func (s *inMemoryServer) URL() string {
+	return s.server.URL
+}
+
+// Close shuts down the server, blocking until all outstanding requests have
+// completed.
+func (s *inMemoryServer) Close() {
+	s.server.Close()
+}
+
+type memoryListener struct {
+	conns  chan net.Conn
+	once   sync.Once
+	closed chan struct{}
+}
+
+// Accept implements net.Listener.
+func (l *memoryListener) Accept() (net.Conn, error) {
+	select {
+	case conn := <-l.conns:
+		return conn, nil
+	case <-l.closed:
+		return nil, errors.New("listener closed")
+	}
+}
+
+// Close implements net.Listener.
+func (l *memoryListener) Close() error {
+	l.once.Do(func() {
+		close(l.closed)
+	})
+	return nil
+}
+
+// Addr implements net.Listener.
+func (l *memoryListener) Addr() net.Addr {
+	return &memoryAddr{}
+}
+
+// DialContext is the type expected by http.Transport.DialContext.
+func (l *memoryListener) DialContext(ctx context.Context, network, addr string) (net.Conn, error) {
+	select {
+	case <-l.closed:
+		return nil, errors.New("listener closed")
+	default:
+	}
+	server, client := net.Pipe()
+	l.conns <- server
+	return client, nil
+}
+
+type memoryAddr struct{}
+
+// Network implements net.Addr.
+func (*memoryAddr) Network() string { return "memory" }
+
+// String implements io.Stringer, returning a value that matches the
+// certificates used by net/http/httptest.
+func (*memoryAddr) String() string { return "example.com" }
diff --git a/protocol/triple/triple_protocol/handler.go b/protocol/triple/triple_protocol/handler.go
new file mode 100644
index 0000000..13a3535
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler.go
@@ -0,0 +1,354 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+)
+
+// A Handler is the server-side implementation of a single RPC defined by a
+// service schema.
+//
+// By default, Handlers support the Connect, gRPC, and gRPC-Web protocols with
+// the binary Protobuf and JSON codecs. They support gzip compression using the
+// standard library's [compress/gzip].
+type Handler struct {
+	spec             Spec
+	implementation   StreamingHandlerFunc
+	protocolHandlers []protocolHandler
+	allowMethod      string // Allow header
+	acceptPost       string // Accept-Post header
+}
+
+// NewUnaryHandler constructs a [Handler] for a request-response procedure.
+func NewUnaryHandler(
+	procedure string,
+	reqInitFunc func() interface{},
+	unary func(context.Context, *Request) (*Response, error),
+	options ...HandlerOption,
+) *Handler {
+	// Wrap the strongly-typed implementation so we can apply interceptors.
+	untyped := UnaryHandlerFunc(func(ctx context.Context, request AnyRequest) (AnyResponse, error) {
+		// verify err
+		if err := ctx.Err(); err != nil {
+			return nil, err
+		}
+		typed, ok := request.(*Request)
+		if !ok {
+			return nil, errorf(CodeInternal, "unexpected handler request type %T", request)
+		}
+		res, err := unary(ctx, typed)
+		if res == nil && err == nil {
+			// This is going to panic during serialization. Debugging is much easier
+			// if we panic here instead, so we can include the procedure name.
+			panic(fmt.Sprintf("%s returned nil *triple.Response and nil error", procedure)) //nolint: forbidigo
+		}
+		return res, err
+	})
+	// todo: modify server func
+	config := newHandlerConfig(procedure, options)
+	if interceptor := config.Interceptor; interceptor != nil {
+		untyped = interceptor.WrapUnaryHandler(untyped)
+	}
+	// receive and send
+	// conn should be responsible for marshal and unmarshal
+	// Given a stream, how should we call the unary function?
+	implementation := func(ctx context.Context, conn StreamingHandlerConn) error {
+		req := reqInitFunc()
+		if err := conn.Receive(req); err != nil {
+			return err
+		}
+		// wrap the specific msg
+		request := NewRequest(req)
+		request.spec = conn.Spec()
+		request.peer = conn.Peer()
+		request.header = conn.RequestHeader()
+		// embed header in context so that user logic could process them via FromIncomingContext
+		ctx = newIncomingContext(ctx, conn.RequestHeader())
+
+		response, err := untyped(ctx, request)
+		if err != nil {
+			return err
+		}
+		// merge headers
+		mergeHeaders(conn.ResponseHeader(), response.Header())
+		mergeHeaders(conn.ResponseTrailer(), response.Trailer())
+		return conn.Send(response.Any())
+	}
+
+	protocolHandlers := config.newProtocolHandlers(StreamTypeUnary)
+	return &Handler{
+		spec:             config.newSpec(StreamTypeUnary),
+		implementation:   implementation,
+		protocolHandlers: protocolHandlers,
+		allowMethod:      sortedAllowMethodValue(protocolHandlers),
+		acceptPost:       sortedAcceptPostValue(protocolHandlers),
+	}
+}
+
+// NewClientStreamHandler constructs a [Handler] for a client streaming procedure.
+func NewClientStreamHandler(
+	procedure string,
+	implementation func(context.Context, *ClientStream) (*Response, error),
+	options ...HandlerOption,
+) *Handler {
+	return newStreamHandler(
+		procedure,
+		StreamTypeClient,
+		func(ctx context.Context, conn StreamingHandlerConn) error {
+			stream := &ClientStream{conn: conn}
+			// embed header in context so that user logic could process them via FromIncomingContext
+			ctx = newIncomingContext(ctx, conn.RequestHeader())
+			res, err := implementation(ctx, stream)
+			if err != nil {
+				return err
+			}
+			if res == nil {
+				// This is going to panic during serialization. Debugging is much easier
+				// if we panic here instead, so we can include the procedure name.
+				panic(fmt.Sprintf("%s returned nil *triple.Response and nil error", procedure)) //nolint: forbidigo
+			}
+			mergeHeaders(conn.ResponseHeader(), res.header)
+			mergeHeaders(conn.ResponseTrailer(), res.trailer)
+			return conn.Send(res.Msg)
+		},
+		options...,
+	)
+}
+
+// NewServerStreamHandler constructs a [Handler] for a server streaming procedure.
+func NewServerStreamHandler(
+	procedure string,
+	reqInitFunc func() interface{},
+	implementation func(context.Context, *Request, *ServerStream) error,
+	options ...HandlerOption,
+) *Handler {
+	return newStreamHandler(
+		procedure,
+		StreamTypeServer,
+		func(ctx context.Context, conn StreamingHandlerConn) error {
+			req := reqInitFunc()
+			if err := conn.Receive(req); err != nil {
+				return err
+			}
+			// embed header in context so that user logic could process them via FromIncomingContext
+			ctx = newIncomingContext(ctx, conn.RequestHeader())
+			return implementation(
+				ctx,
+				&Request{
+					Msg:    req,
+					spec:   conn.Spec(),
+					peer:   conn.Peer(),
+					header: conn.RequestHeader(),
+				},
+				&ServerStream{conn: conn},
+			)
+		},
+		options...,
+	)
+}
+
+// NewBidiStreamHandler constructs a [Handler] for a bidirectional streaming procedure.
+func NewBidiStreamHandler(
+	procedure string,
+	implementation func(context.Context, *BidiStream) error,
+	options ...HandlerOption,
+) *Handler {
+	return newStreamHandler(
+		procedure,
+		StreamTypeBidi,
+		func(ctx context.Context, conn StreamingHandlerConn) error {
+			// embed header in context so that user logic could process them via FromIncomingContext
+			ctx = newIncomingContext(ctx, conn.RequestHeader())
+			return implementation(
+				ctx,
+				&BidiStream{conn: conn},
+			)
+		},
+		options...,
+	)
+}
+
+// ServeHTTP implements [http.Handler].
+func (h *Handler) ServeHTTP(responseWriter http.ResponseWriter, request *http.Request) {
+	// We don't need to defer functions  to close the request body or read to
+	// EOF: the stream we construct later on already does that, and we only
+	// return early when dealing with misbehaving clients. In those cases, it's
+	// okay if we can't re-use the connection.
+	isBidi := (h.spec.StreamType & StreamTypeBidi) == StreamTypeBidi
+	if isBidi && request.ProtoMajor < 2 {
+		// Clients coded to expect full-duplex connections may hang if they've
+		// mistakenly negotiated HTTP/1.1. To unblock them, we must close the
+		// underlying TCP connection.
+		responseWriter.Header().Set("Connection", "close")
+		responseWriter.WriteHeader(http.StatusHTTPVersionNotSupported)
+		return
+	}
+
+	// inspect headers
+	var protocolHandlers []protocolHandler
+	for _, handler := range h.protocolHandlers {
+		if _, ok := handler.Methods()[request.Method]; ok {
+			protocolHandlers = append(protocolHandlers, handler)
+		}
+	}
+
+	if len(protocolHandlers) == 0 {
+		responseWriter.Header().Set("Allow", h.allowMethod)
+		responseWriter.WriteHeader(http.StatusMethodNotAllowed)
+		return
+	}
+
+	contentType := canonicalizeContentType(getHeaderCanonical(request.Header, headerContentType))
+
+	// inspect contentType
+	// Find our implementation of the RPC protocol in use.
+	var protocolHandler protocolHandler
+	for _, handler := range protocolHandlers {
+		if handler.CanHandlePayload(request, contentType) {
+			protocolHandler = handler
+			break
+		}
+	}
+	if protocolHandler == nil {
+		responseWriter.Header().Set("Accept-Post", h.acceptPost)
+		responseWriter.WriteHeader(http.StatusUnsupportedMediaType)
+		return
+	}
+
+	// Establish a stream and serve the RPC.
+	setHeaderCanonical(request.Header, headerContentType, contentType)
+	// process context
+	ctx, cancel, timeoutErr := protocolHandler.SetTimeout(request) //nolint: contextcheck
+	if timeoutErr != nil {
+		ctx = request.Context()
+	}
+	if cancel != nil {
+		defer cancel()
+	}
+	// create stream
+	connCloser, ok := protocolHandler.NewConn(
+		responseWriter,
+		request.WithContext(ctx),
+	)
+	if !ok {
+		// Failed to create stream, usually because client used an unknown
+		// compression algorithm. Nothing further to do.
+		return
+	}
+	if timeoutErr != nil {
+		_ = connCloser.Close(timeoutErr)
+		return
+	}
+	// invoke implementation
+	_ = connCloser.Close(h.implementation(ctx, connCloser))
+}
+
+type handlerConfig struct {
+	CompressionPools             map[string]*compressionPool
+	CompressionNames             []string
+	Codecs                       map[string]Codec
+	CompressMinBytes             int
+	Interceptor                  Interceptor
+	Procedure                    string
+	HandleGRPC                   bool
+	RequireConnectProtocolHeader bool
+	IdempotencyLevel             IdempotencyLevel
+	BufferPool                   *bufferPool
+	ReadMaxBytes                 int
+	SendMaxBytes                 int
+}
+
+func newHandlerConfig(procedure string, options []HandlerOption) *handlerConfig {
+	protoPath := extractProtoPath(procedure)
+	config := handlerConfig{
+		Procedure:        protoPath,
+		CompressionPools: make(map[string]*compressionPool),
+		Codecs:           make(map[string]Codec),
+		HandleGRPC:       true,
+		BufferPool:       newBufferPool(),
+	}
+	withProtoBinaryCodec().applyToHandler(&config)
+	withProtoJSONCodecs().applyToHandler(&config)
+	withGzip().applyToHandler(&config)
+	for _, opt := range options {
+		opt.applyToHandler(&config)
+	}
+	return &config
+}
+
+func (c *handlerConfig) newSpec(streamType StreamType) Spec {
+	return Spec{
+		Procedure:        c.Procedure,
+		StreamType:       streamType,
+		IdempotencyLevel: c.IdempotencyLevel,
+	}
+}
+
+func (c *handlerConfig) newProtocolHandlers(streamType StreamType) []protocolHandler {
+	// initialize protocol
+	var protocols []protocol
+	if streamType == StreamTypeUnary {
+		protocols = append(protocols, &protocolTriple{})
+	}
+	if c.HandleGRPC {
+		protocols = append(protocols, &protocolGRPC{})
+	}
+	// protocol -> protocolHandler
+	handlers := make([]protocolHandler, 0, len(protocols))
+	// initialize codec and compressor
+	codecs := newReadOnlyCodecs(c.Codecs)
+	compressors := newReadOnlyCompressionPools(
+		c.CompressionPools,
+		c.CompressionNames,
+	)
+	for _, protocol := range protocols {
+		handlers = append(handlers, protocol.NewHandler(&protocolHandlerParams{
+			Spec:             c.newSpec(streamType),
+			Codecs:           codecs,
+			CompressionPools: compressors,
+			// config content
+			CompressMinBytes:            c.CompressMinBytes,
+			BufferPool:                  c.BufferPool,
+			ReadMaxBytes:                c.ReadMaxBytes,
+			SendMaxBytes:                c.SendMaxBytes,
+			RequireTripleProtocolHeader: c.RequireConnectProtocolHeader,
+			IdempotencyLevel:            c.IdempotencyLevel,
+		}))
+	}
+	return handlers
+}
+
+func newStreamHandler(
+	procedure string,
+	streamType StreamType,
+	implementation StreamingHandlerFunc,
+	options ...HandlerOption,
+) *Handler {
+	config := newHandlerConfig(procedure, options)
+	if ic := config.Interceptor; ic != nil {
+		implementation = ic.WrapStreamingHandler(implementation)
+	}
+	protocolHandlers := config.newProtocolHandlers(streamType)
+	return &Handler{
+		spec:             config.newSpec(streamType),
+		implementation:   implementation,
+		protocolHandlers: protocolHandlers,
+		allowMethod:      sortedAllowMethodValue(protocolHandlers),
+		acceptPost:       sortedAcceptPostValue(protocolHandlers),
+	}
+}
diff --git a/protocol/triple/triple_protocol/handler_compat.go b/protocol/triple/triple_protocol/handler_compat.go
new file mode 100644
index 0000000..e83cddb
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler_compat.go
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple_protocol
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+)
+
+import (
+	"github.com/dubbogo/grpc-go"
+)
+
+import (
+	dubbo_protocol "dubbo.apache.org/dubbo-go/v3/protocol"
+)
+
+type MethodHandler func(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error)
+
+type tripleCompatInterceptor struct {
+	spec        Spec
+	peer        Peer
+	header      http.Header
+	procedure   string
+	interceptor Interceptor
+}
+
+// be compatible with old triple-gen code
+func (t *tripleCompatInterceptor) compatUnaryServerInterceptor(ctx context.Context, req interface{}, info *grpc.UnaryServerInfo, handler grpc.UnaryHandler) (interface{}, error) {
+	request := NewRequest(req)
+	request.spec = t.spec
+	request.peer = t.peer
+	request.header = t.header
+
+	unaryFunc := func(ctx context.Context, request AnyRequest) (AnyResponse, error) {
+		if err := ctx.Err(); err != nil {
+			return nil, err
+		}
+		typed, ok := request.(*Request)
+		if !ok {
+			return nil, errorf(CodeInternal, "unexpected handler request type %T", request)
+		}
+		respRaw, err := handler(ctx, typed.Any())
+		if respRaw == nil && err == nil {
+			// This is going to panic during serialization. Debugging is much easier
+			// if we panic here instead, so we can include the procedure name.
+			panic(fmt.Sprintf("%s returned nil resp and nil error", t.procedure)) //nolint: forbidigo
+		}
+		resp, ok := respRaw.(*dubbo_protocol.RPCResult)
+		if !ok {
+			panic(fmt.Sprintf("%+v is not of type *RPCResult", respRaw))
+		}
+		// todo(DMwangnima): expose API for users to write response headers and trailers
+		return NewResponse(resp.Rest), err
+	}
+
+	if t.interceptor != nil {
+		unaryFunc = t.interceptor.WrapUnaryHandler(unaryFunc)
+	}
+
+	return unaryFunc(ctx, request)
+}
+
+func NewCompatUnaryHandler(
+	procedure string,
+	srv interface{},
+	unary MethodHandler,
+	options ...HandlerOption,
+) *Handler {
+	config := newHandlerConfig(procedure, options)
+
+	implementation := func(ctx context.Context, conn StreamingHandlerConn) error {
+		compatInterceptor := &tripleCompatInterceptor{
+			spec:        conn.Spec(),
+			peer:        conn.Peer(),
+			header:      conn.RequestHeader(),
+			procedure:   config.Procedure,
+			interceptor: config.Interceptor,
+		}
+		decodeFunc := func(req interface{}) error {
+			if err := conn.Receive(req); err != nil {
+				return err
+			}
+			return nil
+		}
+		ctx = context.WithValue(ctx, "XXX_TRIPLE_GO_INTERFACE_NAME", config.Procedure)
+		respRaw, err := unary(srv, ctx, decodeFunc, compatInterceptor.compatUnaryServerInterceptor)
+		if err != nil {
+			return err
+		}
+		resp := respRaw.(*Response)
+		// merge headers
+		mergeHeaders(conn.ResponseHeader(), resp.Header())
+		mergeHeaders(conn.ResponseTrailer(), resp.Trailer())
+		return conn.Send(resp.Any())
+	}
+
+	protocolHandlers := config.newProtocolHandlers(StreamTypeUnary)
+	return &Handler{
+		spec:             config.newSpec(StreamTypeUnary),
+		implementation:   implementation,
+		protocolHandlers: protocolHandlers,
+		allowMethod:      sortedAllowMethodValue(protocolHandlers),
+		acceptPost:       sortedAcceptPostValue(protocolHandlers),
+	}
+}
diff --git a/protocol/triple/triple_protocol/handler_example_test.go b/protocol/triple/triple_protocol/handler_example_test.go
new file mode 100644
index 0000000..ca3d144
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler_example_test.go
@@ -0,0 +1,72 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"net/http"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+// ExamplePingServer implements some trivial business logic. The Protobuf
+// definition for this API is in proto/triple/ping/v1/ping.proto.
+type ExamplePingServer struct {
+	pingv1connect.UnimplementedPingServiceHandler
+}
+
+// Ping implements pingv1connect.PingServiceHandler.
+func (*ExamplePingServer) Ping(
+	_ context.Context,
+	request *triple.Request,
+) (*triple.Response, error) {
+	msg := request.Msg.(*pingv1.PingRequest)
+	return triple.NewResponse(&pingv1.PingResponse{
+		Number: msg.Number,
+		Text:   msg.Text,
+	}), nil
+}
+
+func Example_handler() {
+	// protoc-gen-triple-go generates constructors that return plain net/http
+	// Handlers, so they're compatible with most Go HTTP routers and middleware
+	// (for example, net/http's StripPrefix). Each handler automatically supports
+	// the Connect, gRPC, and gRPC-Web protocols.
+	mux := http.NewServeMux()
+	mux.Handle(
+		pingv1connect.NewPingServiceHandler(
+			&ExamplePingServer{}, // our business logic
+		),
+	)
+	// You can serve gRPC's health and server reflection APIs using
+	// github.com/bufbuild/triple-grpchealth-go and
+	// github.com/bufbuild/triple-grpcreflect-go.
+	_ = http.ListenAndServeTLS(
+		"localhost:8080",
+		"internal/testdata/server.crt",
+		"internal/testdata/server.key",
+		mux,
+	)
+	// To serve HTTP/2 requests without TLS (as many gRPC clients expect), import
+	// golang.org/x/net/http2/h2c and golang.org/x/net/http2 and change to:
+	// _ = http.ListenAndServe(
+	// 	"localhost:8080",
+	// 	h2c.NewHandler(mux, &http2.Server{}),
+	// )
+}
diff --git a/protocol/triple/triple_protocol/handler_ext_test.go b/protocol/triple/triple_protocol/handler_ext_test.go
new file mode 100644
index 0000000..80f0a6e
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler_ext_test.go
@@ -0,0 +1,200 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"encoding/json"
+	"net/http"
+	"net/http/httptest"
+	"strings"
+	"testing"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func TestHandler_ServeHTTP(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		successPingServer{},
+	))
+	const pingProcedure = "/" + pingv1connect.PingServiceName + "/Ping"
+	const sumProcedure = "/" + pingv1connect.PingServiceName + "/Sum"
+	server := httptest.NewServer(mux)
+	client := server.Client()
+	t.Cleanup(func() {
+		server.Close()
+	})
+
+	t.Run("get_method_no_encoding", func(t *testing.T) {
+		t.Parallel()
+		request, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure,
+			strings.NewReader(""),
+		)
+		assert.Nil(t, err)
+		resp, err := client.Do(request)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusUnsupportedMediaType)
+	})
+
+	t.Run("get_method_bad_encoding", func(t *testing.T) {
+		t.Parallel()
+		request, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure+`?encoding=unk&message={}`,
+			strings.NewReader(""),
+		)
+		assert.Nil(t, err)
+		resp, err := client.Do(request)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusUnsupportedMediaType)
+	})
+
+	//triple并不支持POST
+	//t.Run("idempotent_get_method", func(t *testing.T) {
+	//	t.Parallel()
+	//	request, err := http.NewRequestWithContext(
+	//		context.Background(),
+	//		http.MethodPost,
+	//		server.URL+pingProcedure+`?encoding=json&message={}`,
+	//		strings.NewReader(""),
+	//	)
+	//	assert.Nil(t, err)
+	//	resp, err := client.Do(request)
+	//	assert.Nil(t, err)
+	//	defer resp.Body.Close()
+	//	assert.Equal(t, resp.StatusCode, http.StatusOK)
+	//})
+
+	//不支持流式调用
+	//t.Run("method_not_allowed", func(t *testing.T) {
+	//	t.Parallel()
+	//	request, err := http.NewRequestWithContext(
+	//		context.Background(),
+	//		http.MethodGet,
+	//		server.URL+sumProcedure,
+	//		strings.NewReader(""),
+	//	)
+	//	assert.Nil(t, err)
+	//	resp, err := client.Do(request)
+	//	assert.Nil(t, err)
+	//	defer resp.Body.Close()
+	//	assert.Equal(t, resp.StatusCode, http.StatusMethodNotAllowed)
+	//	assert.Equal(t, resp.Header.Get("Allow"), http.MethodPost)
+	//})
+
+	t.Run("unsupported_content_type", func(t *testing.T) {
+		t.Parallel()
+		request, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure,
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		request.Header.Set("Content-Type", "application/x-custom-json")
+		resp, err := client.Do(request)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusUnsupportedMediaType)
+		assert.Equal(t, resp.Header.Get("Accept-Post"), strings.Join([]string{
+			"application/grpc",
+			"application/grpc+json",
+			"application/grpc+json; charset=utf-8",
+			"application/grpc+proto",
+			"application/json",
+			"application/json; charset=utf-8",
+			"application/proto",
+		}, ", "))
+	})
+
+	t.Run("charset_in_content_type_header", func(t *testing.T) {
+		t.Parallel()
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure,
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/json;Charset=Utf-8")
+		resp, err := client.Do(req)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusOK)
+	})
+
+	t.Run("unsupported_charset", func(t *testing.T) {
+		t.Parallel()
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure,
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/json; charset=shift-jis")
+		resp, err := client.Do(req)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusUnsupportedMediaType)
+	})
+
+	t.Run("unsupported_content_encoding", func(t *testing.T) {
+		t.Parallel()
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+pingProcedure,
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/json")
+		req.Header.Set("Content-Encoding", "invalid")
+		resp, err := client.Do(req)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, resp.StatusCode, http.StatusNotFound)
+
+		type errorMessage struct {
+			Code    string `json:"code,omitempty"`
+			Message string `json:"message,omitempty"`
+		}
+		var message errorMessage
+		err = json.NewDecoder(resp.Body).Decode(&message)
+		assert.Nil(t, err)
+		assert.Equal(t, message.Message, `unknown compression "invalid": supported encodings are gzip`)
+		assert.Equal(t, message.Code, triple.CodeUnimplemented.String())
+	})
+}
+
+type successPingServer struct {
+	pingv1connect.UnimplementedPingServiceHandler
+}
+
+func (successPingServer) Ping(context.Context, *triple.Request) (*triple.Response, error) {
+	return &triple.Response{}, nil
+}
diff --git a/protocol/triple/triple_protocol/handler_stream.go b/protocol/triple/triple_protocol/handler_stream.go
new file mode 100644
index 0000000..c537089
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler_stream.go
@@ -0,0 +1,189 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"errors"
+	"io"
+	"net/http"
+)
+
+// ClientStream is the handler's view of a client streaming RPC.
+//
+// It's constructed as part of [Handler] invocation, but doesn't currently have
+// an exported constructor.
+type ClientStream struct {
+	conn StreamingHandlerConn
+	msg  interface{}
+	err  error
+}
+
+// Spec returns the specification for the RPC.
+func (c *ClientStream) Spec() Spec {
+	return c.conn.Spec()
+}
+
+// Peer describes the client for this RPC.
+func (c *ClientStream) Peer() Peer {
+	return c.conn.Peer()
+}
+
+// RequestHeader returns the headers received from the client.
+func (c *ClientStream) RequestHeader() http.Header {
+	return c.conn.RequestHeader()
+}
+
+// Receive advances the stream to the next message, which will then be
+// available through the Msg method. It returns false when the stream stops,
+// either by reaching the end or by encountering an unexpected error. After
+// Receive returns false, the Err method will return any unexpected error
+// encountered.
+func (c *ClientStream) Receive(msg interface{}) bool {
+	if c.err != nil {
+		return false
+	}
+	c.msg = msg
+	c.err = c.conn.Receive(c.msg)
+	return c.err == nil
+}
+
+// Msg returns the most recent message unmarshaled by a call to Receive.
+func (c *ClientStream) Msg() interface{} {
+	// todo:// process nil pointer
+	//if c.msg == nil {
+	//	c.msg = new(Req)
+	//}
+	return c.msg
+}
+
+// Err returns the first non-EOF error that was encountered by Receive.
+func (c *ClientStream) Err() error {
+	if c.err == nil || errors.Is(c.err, io.EOF) {
+		return nil
+	}
+	return c.err
+}
+
+// Conn exposes the underlying StreamingHandlerConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (c *ClientStream) Conn() StreamingHandlerConn {
+	return c.conn
+}
+
+// ServerStream is the handler's view of a server streaming RPC.
+//
+// It's constructed as part of [Handler] invocation, but doesn't currently have
+// an exported constructor.
+type ServerStream struct {
+	conn StreamingHandlerConn
+}
+
+// ResponseHeader returns the response headers. Headers are sent with the first
+// call to Send.
+//
+// Headers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (s *ServerStream) ResponseHeader() http.Header {
+	return s.conn.ResponseHeader()
+}
+
+// ResponseTrailer returns the response trailers. Handlers may write to the
+// response trailers at any time before returning.
+//
+// Trailers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (s *ServerStream) ResponseTrailer() http.Header {
+	return s.conn.ResponseTrailer()
+}
+
+// Send a message to the client. The first call to Send also sends the response
+// headers.
+func (s *ServerStream) Send(msg interface{}) error {
+	if msg == nil {
+		return s.conn.Send(nil)
+	}
+	return s.conn.Send(msg)
+}
+
+// Conn exposes the underlying StreamingHandlerConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (s *ServerStream) Conn() StreamingHandlerConn {
+	return s.conn
+}
+
+// BidiStream is the handler's view of a bidirectional streaming RPC.
+//
+// It's constructed as part of [Handler] invocation, but doesn't currently have
+// an exported constructor.
+type BidiStream struct {
+	conn StreamingHandlerConn
+}
+
+// Spec returns the specification for the RPC.
+func (b *BidiStream) Spec() Spec {
+	return b.conn.Spec()
+}
+
+// Peer describes the client for this RPC.
+func (b *BidiStream) Peer() Peer {
+	return b.conn.Peer()
+}
+
+// RequestHeader returns the headers received from the client.
+func (b *BidiStream) RequestHeader() http.Header {
+	return b.conn.RequestHeader()
+}
+
+// Receive a message. When the client is done sending messages, Receive will
+// return an error that wraps [io.EOF].
+func (b *BidiStream) Receive(msg interface{}) error {
+	if err := b.conn.Receive(msg); err != nil {
+		return err
+	}
+	return nil
+}
+
+// ResponseHeader returns the response headers. Headers are sent with the first
+// call to Send.
+//
+// Headers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (b *BidiStream) ResponseHeader() http.Header {
+	return b.conn.ResponseHeader()
+}
+
+// ResponseTrailer returns the response trailers. Handlers may write to the
+// response trailers at any time before returning.
+//
+// Trailers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols. Applications shouldn't write them.
+func (b *BidiStream) ResponseTrailer() http.Header {
+	return b.conn.ResponseTrailer()
+}
+
+// Send a message to the client. The first call to Send also sends the response
+// headers.
+func (b *BidiStream) Send(msg interface{}) error {
+	if msg == nil {
+		return b.conn.Send(nil)
+	}
+	return b.conn.Send(msg)
+}
+
+// Conn exposes the underlying StreamingHandlerConn. This may be useful if
+// you'd prefer to wrap the connection in a different high-level API.
+func (b *BidiStream) Conn() StreamingHandlerConn {
+	return b.conn
+}
diff --git a/protocol/triple/triple_protocol/handler_stream_compat.go b/protocol/triple/triple_protocol/handler_stream_compat.go
new file mode 100644
index 0000000..9cac7b5
--- /dev/null
+++ b/protocol/triple/triple_protocol/handler_stream_compat.go
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple_protocol
+
+import (
+	"context"
+)
+
+import (
+	"github.com/dubbogo/grpc-go"
+	"github.com/dubbogo/grpc-go/metadata"
+)
+
+type compatHandlerStream struct {
+	ctx  context.Context
+	conn StreamingHandlerConn
+}
+
+func (c *compatHandlerStream) SetHeader(md metadata.MD) error {
+	// todo(DMwangnima): add header method for streaming
+	return nil
+}
+
+func (c *compatHandlerStream) SendHeader(md metadata.MD) error {
+	// todo(DMwangnima): add header method for streaming
+	return nil
+}
+
+func (c *compatHandlerStream) SetTrailer(md metadata.MD) {
+	// todo(DMwangnima): add trailer method for streaming
+	return
+}
+
+func (c *compatHandlerStream) Context() context.Context {
+	return c.ctx
+}
+
+func (c *compatHandlerStream) SendMsg(m interface{}) error {
+	return c.conn.Send(m)
+}
+
+func (c *compatHandlerStream) RecvMsg(m interface{}) error {
+	return c.conn.Receive(m)
+}
+
+func NewCompatStreamHandler(
+	procedure string,
+	srv interface{},
+	typ StreamType,
+	implementation func(srv interface{}, stream grpc.ServerStream) error,
+	options ...HandlerOption,
+) *Handler {
+	return newStreamHandler(
+		procedure,
+		typ,
+		func(ctx context.Context, conn StreamingHandlerConn) error {
+			stream := &compatHandlerStream{
+				ctx:  ctx,
+				conn: conn,
+			}
+			return implementation(srv, stream)
+		},
+		options...,
+	)
+}
diff --git a/protocol/triple/triple_protocol/header.go b/protocol/triple/triple_protocol/header.go
new file mode 100644
index 0000000..261a6ad
--- /dev/null
+++ b/protocol/triple/triple_protocol/header.go
@@ -0,0 +1,174 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"encoding/base64"
+	"fmt"
+	"net/http"
+)
+
+// EncodeBinaryHeader base64-encodes the data. It always emits unpadded values.
+//
+// In the Triple, gRPC, and gRPC-Web protocols, binary headers must have keys
+// ending in "-Bin".
+func EncodeBinaryHeader(data []byte) string {
+	// gRPC specification says that implementations should emit unpadded values.
+	return base64.RawStdEncoding.EncodeToString(data)
+}
+
+// DecodeBinaryHeader base64-decodes the data. It can decode padded or unpadded
+// values. Following usual HTTP semantics, multiple base64-encoded values may
+// be joined with a comma. When receiving such comma-separated values, split
+// them with [strings.Split] before calling DecodeBinaryHeader.
+//
+// Binary headers sent using the Triple, gRPC, and gRPC-Web protocols have
+// keys ending in "-Bin".
+func DecodeBinaryHeader(data string) ([]byte, error) {
+	if len(data)%4 != 0 {
+		// Data definitely isn't padded.
+		return base64.RawStdEncoding.DecodeString(data)
+	}
+	// Either the data was padded, or padding wasn't necessary. In both cases,
+	// the padding-aware decoder works.
+	return base64.StdEncoding.DecodeString(data)
+}
+
+func mergeHeaders(into, from http.Header) {
+	for k, vals := range from {
+		into[k] = append(into[k], vals...)
+	}
+}
+
+// getCanonicalHeader is a shortcut for Header.Get() which
+// bypasses the CanonicalMIMEHeaderKey operation when we
+// know the key is already in canonical form.
+func getHeaderCanonical(h http.Header, key string) string {
+	if h == nil {
+		return ""
+	}
+	v := h[key]
+	if len(v) == 0 {
+		return ""
+	}
+	return v[0]
+}
+
+// setHeaderCanonical is a shortcut for Header.Set() which
+// bypasses the CanonicalMIMEHeaderKey operation when we
+// know the key is already in canonical form.
+func setHeaderCanonical(h http.Header, key, value string) {
+	h[key] = []string{value}
+}
+
+// delHeaderCanonical is a shortcut for Header.Del() which
+// bypasses the CanonicalMIMEHeaderKey operation when we
+// know the key is already in canonical form.
+func delHeaderCanonical(h http.Header, key string) {
+	delete(h, key)
+}
+
+// addHeaderCanonical is a shortcut for Header.Add() which
+// bypasses the CanonicalMIMEHeaderKey operation when we
+// know the key is already in canonical form.
+func addHeaderCanonical(h http.Header, key, value string) {
+	h[key] = append(h[key], value)
+}
+
+type headerIncomingKey struct{}
+type headerOutgoingKey struct{}
+type handlerOutgoingKey struct{}
+
+func newIncomingContext(ctx context.Context, header http.Header) context.Context {
+	return context.WithValue(ctx, headerIncomingKey{}, header)
+}
+
+// NewOutgoingContext sets headers entirely. If there are existing headers, they would be replaced.
+// It is used for passing headers to server-side.
+// It is like grpc.NewOutgoingContext.
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#sending-metadata.
+func NewOutgoingContext(ctx context.Context, header http.Header) context.Context {
+	return context.WithValue(ctx, headerOutgoingKey{}, header)
+}
+
+// AppendToOutgoingContext merges kv pairs from user and existing headers.
+// It is used for passing headers to server-side.
+// It is like grpc.AppendToOutgoingContext.
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#sending-metadata.
+func AppendToOutgoingContext(ctx context.Context, kv ...string) context.Context {
+	if len(kv)%2 == 1 {
+		panic(fmt.Sprintf("AppendToOutgoingContext got an odd number of input pairs for header: %d", len(kv)))
+	}
+	var header http.Header
+	headerRaw := ctx.Value(headerOutgoingKey{})
+	if headerRaw == nil {
+		header = make(http.Header)
+	} else {
+		header = headerRaw.(http.Header)
+	}
+	for i := 0; i < len(kv); i += 2 {
+		// todo(DMwangnima): think about lowering
+		header.Add(kv[i], kv[i+1])
+	}
+	return context.WithValue(ctx, headerOutgoingKey{}, header)
+}
+
+// FromIncomingContext retrieves headers passed by client-side. It is like grpc.FromIncomingContext.
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#receiving-metadata-1.
+func FromIncomingContext(ctx context.Context) (http.Header, bool) {
+	header, ok := ctx.Value(headerIncomingKey{}).(http.Header)
+	if !ok {
+		return nil, false
+	}
+	return header, true
+}
+
+// SetHeader is used for setting response header in server-side. It is like grpc.SendHeader(ctx, header) but
+// not send header.
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#unary-call-2.
+func SetHeader(ctx context.Context, header http.Header) error {
+	conn, ok := ctx.Value(handlerOutgoingKey{}).(StreamingHandlerConn)
+	if !ok {
+		// todo(DMwangnima): return standard error
+		return fmt.Errorf("triple: failed to fetch the connection from the context %v", ctx)
+	}
+	mergeHeaders(conn.ResponseHeader(), header)
+	return nil
+}
+
+// SetTrailer is used for setting response trailers in server-side. It is like grpc.SetTrailer(ctx, header).
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#unary-call-2.
+func SetTrailer(ctx context.Context, trailer http.Header) error {
+	conn, ok := ctx.Value(handlerOutgoingKey{}).(StreamingHandlerConn)
+	if !ok {
+		// todo(DMwangnima): return standard error
+		return fmt.Errorf("triple: failed to fetch the connection from the context %v", ctx)
+	}
+	mergeHeaders(conn.ResponseTrailer(), trailer)
+	return nil
+}
+
+// SendHeader is used for setting response headers in server-side and send them directly. It is like grpc.SendHeader(ctx, header).
+// Please refer to https://github.com/grpc/grpc-go/blob/master/Documentation/grpc-metadata.md#unary-call-2.
+func SendHeader(ctx context.Context, header http.Header) error {
+	conn, ok := ctx.Value(handlerOutgoingKey{}).(StreamingHandlerConn)
+	if !ok {
+		// todo(DMwangnima): return standard error
+		return fmt.Errorf("triple: failed to fetch the connection from the context %v", ctx)
+	}
+	mergeHeaders(conn.RequestHeader(), header)
+	return conn.Send(nil)
+}
diff --git a/protocol/triple/triple_protocol/header_test.go b/protocol/triple/triple_protocol/header_test.go
new file mode 100644
index 0000000..3b73b35
--- /dev/null
+++ b/protocol/triple/triple_protocol/header_test.go
@@ -0,0 +1,60 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"net/http"
+	"testing"
+	"testing/quick"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestBinaryEncodingQuick(t *testing.T) {
+	t.Parallel()
+	roundtrip := func(binary []byte) bool {
+		encoded := EncodeBinaryHeader(binary)
+		decoded, err := DecodeBinaryHeader(encoded)
+		if err != nil {
+			// We want to abort immediately. Don't use our assert package.
+			t.Fatalf("decode error: %v", err)
+		}
+		return bytes.Equal(decoded, binary)
+	}
+	if err := quick.Check(roundtrip, nil /* config */); err != nil {
+		t.Error(err)
+	}
+}
+
+func TestHeaderMerge(t *testing.T) {
+	t.Parallel()
+	header := http.Header{
+		"Foo": []string{"one"},
+	}
+	mergeHeaders(header, http.Header{
+		"Foo": []string{"two"},
+		"Bar": []string{"one"},
+		"Baz": nil,
+	})
+	expect := http.Header{
+		"Foo": []string{"one", "two"},
+		"Bar": []string{"one"},
+		"Baz": nil,
+	}
+	assert.Equal(t, header, expect)
+}
diff --git a/protocol/triple/triple_protocol/idempotency_level.go b/protocol/triple/triple_protocol/idempotency_level.go
new file mode 100644
index 0000000..ba20428
--- /dev/null
+++ b/protocol/triple/triple_protocol/idempotency_level.go
@@ -0,0 +1,70 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"fmt"
+)
+
+// An IdempotencyLevel is a value that declares how "idempotent" an RPC is. This
+// value can affect RPC behaviors, such as determining whether it is safe to
+// retry a request, or what kinds of request modalities are allowed for a given
+// procedure.
+type IdempotencyLevel int
+
+// NOTE: For simplicity, these should be kept in sync with the values of the
+// google.protobuf.MethodOptions.IdempotencyLevel enumeration.
+
+const (
+	// IdempotencyUnknown is the default idempotency level. A procedure with
+	// this idempotency level may not be idempotent. This is appropriate for
+	// any kind of procedure.
+	IdempotencyUnknown IdempotencyLevel = 0
+
+	// IdempotencyNoSideEffects is the idempotency level that specifies that a
+	// given call has no side-effects. This is equivalent to [RFC 9110 § 9.2.1]
+	// "safe" methods in terms of semantics. This procedure should not mutate
+	// any state. This idempotency level is appropriate for queries, or anything
+	// that would be suitable for an HTTP GET request. In addition, due to the
+	// lack of side-effects, such a procedure would be suitable to retry and
+	// expect that the results will not be altered by preceding attempts.
+	//
+	// [RFC 9110 § 9.2.1]: https://www.rfc-editor.org/rfc/rfc9110.html#section-9.2.1
+	IdempotencyNoSideEffects IdempotencyLevel = 1
+
+	// IdempotencyIdempotent is the idempotency level that specifies that a
+	// given call is "idempotent", such that multiple instances of the same
+	// request to this procedure would have the same side-effects as a single
+	// request. This is equivalent to [RFC 9110 § 9.2.2] "idempotent" methods.
+	// This level is a subset of the previous level. This idempotency level is
+	// appropriate for any procedure that is safe to retry multiple times
+	// and be guaranteed that the response and side-effects will not be altered
+	// as a result of multiple attempts, for example, entity deletion requests.
+	//
+	// [RFC 9110 § 9.2.2]: https://www.rfc-editor.org/rfc/rfc9110.html#section-9.2.2
+	IdempotencyIdempotent IdempotencyLevel = 2
+)
+
+func (i IdempotencyLevel) String() string {
+	switch i {
+	case IdempotencyUnknown:
+		return "idempotency_unknown"
+	case IdempotencyNoSideEffects:
+		return "no_side_effects"
+	case IdempotencyIdempotent:
+		return "idempotent"
+	}
+	return fmt.Sprintf("idempotency_%d", i)
+}
diff --git a/protocol/triple/triple_protocol/interceptor.go b/protocol/triple/triple_protocol/interceptor.go
new file mode 100644
index 0000000..5ad7f56
--- /dev/null
+++ b/protocol/triple/triple_protocol/interceptor.go
@@ -0,0 +1,124 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+)
+
+// UnaryFunc is the generic signature of a unary RPC. Interceptors may wrap
+// Funcs.
+//
+// The type of the request and response structs depend on the codec being used.
+// When using Protobuf, request.Any() and response.Any() will always be
+// [proto.Message] implementations.
+type UnaryFunc func(context.Context, AnyRequest, AnyResponse) error
+
+// UnaryHandlerFunc is the generic signature of a unary RPC from the handler's
+// perspective. Interceptors may wrap UnaryHandlerFuncs.
+type UnaryHandlerFunc func(ctx context.Context, request AnyRequest) (AnyResponse, error)
+
+// StreamingClientFunc is the generic signature of a streaming RPC from the client's
+// perspective. Interceptors may wrap StreamingClientFuncs.
+type StreamingClientFunc func(context.Context, Spec) StreamingClientConn
+
+// StreamingHandlerFunc is the generic signature of a streaming RPC from the
+// handler's perspective. Interceptors may wrap StreamingHandlerFuncs.
+type StreamingHandlerFunc func(context.Context, StreamingHandlerConn) error
+
+// An Interceptor adds logic to a generated handler or client, like the
+// decorators or middleware you may have seen in other libraries. Interceptors
+// may replace the context, mutate requests and responses, handle errors,
+// retry, recover from panics, emit logs and metrics, or do nearly anything
+// else.
+//
+// The returned functions must be safe to call concurrently.
+type Interceptor interface {
+	WrapUnary(UnaryFunc) UnaryFunc
+	WrapUnaryHandler(UnaryHandlerFunc) UnaryHandlerFunc
+	WrapStreamingClient(StreamingClientFunc) StreamingClientFunc
+	WrapStreamingHandler(StreamingHandlerFunc) StreamingHandlerFunc
+}
+
+// UnaryInterceptorFunc is a simple Interceptor implementation that only
+// wraps unary RPCs from client's perspective. It has no effect on server side unary RPC and streaming RPCs.
+type UnaryInterceptorFunc func(UnaryFunc) UnaryFunc
+
+// WrapUnary implements [Interceptor] by applying the interceptor function.
+func (f UnaryInterceptorFunc) WrapUnary(next UnaryFunc) UnaryFunc {
+	return f(next)
+}
+
+// WrapUnaryHandler implements [Interceptor] with a no-op.
+func (f UnaryInterceptorFunc) WrapUnaryHandler(next UnaryHandlerFunc) UnaryHandlerFunc {
+	return next
+}
+
+// WrapStreamingClient implements [Interceptor] with a no-op.
+func (f UnaryInterceptorFunc) WrapStreamingClient(next StreamingClientFunc) StreamingClientFunc {
+	return next
+}
+
+// WrapStreamingHandler implements [Interceptor] with a no-op.
+func (f UnaryInterceptorFunc) WrapStreamingHandler(next StreamingHandlerFunc) StreamingHandlerFunc {
+	return next
+}
+
+// A chain composes multiple interceptors into one.
+type chain struct {
+	interceptors []Interceptor
+}
+
+// newChain composes multiple interceptors into one.
+func newChain(interceptors []Interceptor) *chain {
+	// We usually wrap in reverse order to have the first interceptor from
+	// the slice act first. Rather than doing this dance repeatedly, reverse the
+	// interceptor order now.
+	var chain chain
+	for i := len(interceptors) - 1; i >= 0; i-- {
+		if interceptor := interceptors[i]; interceptor != nil {
+			chain.interceptors = append(chain.interceptors, interceptor)
+		}
+	}
+	return &chain
+}
+
+func (c *chain) WrapUnary(next UnaryFunc) UnaryFunc {
+	for _, interceptor := range c.interceptors {
+		next = interceptor.WrapUnary(next)
+	}
+	return next
+}
+
+func (c *chain) WrapUnaryHandler(next UnaryHandlerFunc) UnaryHandlerFunc {
+	for _, interceptor := range c.interceptors {
+		next = interceptor.WrapUnaryHandler(next)
+	}
+	return next
+}
+
+func (c *chain) WrapStreamingClient(next StreamingClientFunc) StreamingClientFunc {
+	for _, interceptor := range c.interceptors {
+		next = interceptor.WrapStreamingClient(next)
+	}
+	return next
+}
+
+func (c *chain) WrapStreamingHandler(next StreamingHandlerFunc) StreamingHandlerFunc {
+	for _, interceptor := range c.interceptors {
+		next = interceptor.WrapStreamingHandler(next)
+	}
+	return next
+}
diff --git a/protocol/triple/triple_protocol/interceptor_example_test.go b/protocol/triple/triple_protocol/interceptor_example_test.go
new file mode 100644
index 0000000..cfbbdc9
--- /dev/null
+++ b/protocol/triple/triple_protocol/interceptor_example_test.go
@@ -0,0 +1,99 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"log"
+	"os"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func ExampleUnaryInterceptorFunc() {
+	logger := log.New(os.Stdout, "" /* prefix */, 0 /* flags */)
+	loggingInterceptor := triple.UnaryInterceptorFunc(
+		func(next triple.UnaryFunc) triple.UnaryFunc {
+			return triple.UnaryFunc(func(ctx context.Context, request triple.AnyRequest, response triple.AnyResponse) error {
+				logger.Println("calling:", request.Spec().Procedure)
+				logger.Println("request:", request.Any())
+				err := next(ctx, request, response)
+				if err != nil {
+					logger.Println("error:", err)
+				} else {
+					logger.Println("response:", response.Any())
+				}
+				return err
+			})
+		},
+	)
+	client := pingv1connect.NewPingServiceClient(
+		examplePingServer.Client(),
+		examplePingServer.URL(),
+		triple.WithInterceptors(loggingInterceptor),
+	)
+	if err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{Number: 42}), triple.NewResponse(&pingv1.PingResponse{})); err != nil {
+		logger.Println("error:", err)
+		return
+	}
+
+	// Output:
+	// calling: /connect.ping.v1.PingService/Ping
+	// request: number:42
+	// response: number:42
+}
+
+func ExampleWithInterceptors() {
+	logger := log.New(os.Stdout, "" /* prefix */, 0 /* flags */)
+	outer := triple.UnaryInterceptorFunc(
+		func(next triple.UnaryFunc) triple.UnaryFunc {
+			return triple.UnaryFunc(func(ctx context.Context, req triple.AnyRequest, res triple.AnyResponse) error {
+				logger.Println("outer interceptor: before call")
+				err := next(ctx, req, res)
+				logger.Println("outer interceptor: after call")
+				return err
+			})
+		},
+	)
+	inner := triple.UnaryInterceptorFunc(
+		func(next triple.UnaryFunc) triple.UnaryFunc {
+			return triple.UnaryFunc(func(ctx context.Context, req triple.AnyRequest, res triple.AnyResponse) error {
+				logger.Println("inner interceptor: before call")
+				err := next(ctx, req, res)
+				logger.Println("inner interceptor: after call")
+				return err
+			})
+		},
+	)
+	client := pingv1connect.NewPingServiceClient(
+		examplePingServer.Client(),
+		examplePingServer.URL(),
+		triple.WithInterceptors(outer, inner),
+	)
+	if err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{})); err != nil {
+		logger.Println("error:", err)
+		return
+	}
+
+	// Output:
+	// outer interceptor: before call
+	// inner interceptor: before call
+	// inner interceptor: after call
+	// outer interceptor: after call
+}
diff --git a/protocol/triple/triple_protocol/interceptor_ext_test.go b/protocol/triple/triple_protocol/interceptor_ext_test.go
new file mode 100644
index 0000000..d3b6ed4
--- /dev/null
+++ b/protocol/triple/triple_protocol/interceptor_ext_test.go
@@ -0,0 +1,285 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"net/http"
+	"net/http/httptest"
+	"testing"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+func TestOnionOrderingEndToEnd(t *testing.T) {
+	t.Parallel()
+	// Helper function: returns a function that asserts that there's some value
+	// set for header "expect", and adds a value for header "add".
+	newInspector := func(expect, add string) func(triple.Spec, http.Header) {
+		return func(spec triple.Spec, header http.Header) {
+			if expect != "" {
+				assert.NotZero(
+					t,
+					header.Get(expect),
+					assert.Sprintf(
+						"%s (IsClient %v): header %q missing: %v",
+						spec.Procedure,
+						spec.IsClient,
+						expect,
+						header,
+					),
+				)
+			}
+			header.Set(add, "v")
+		}
+	}
+	// Helper function: asserts that there's a value present for header keys
+	// "one", "two", "three", and "four".
+	assertAllPresent := func(spec triple.Spec, header http.Header) {
+		for _, key := range []string{"one", "two", "three", "four"} {
+			assert.NotZero(
+				t,
+				header.Get(key),
+				assert.Sprintf(
+					"%s (IsClient %v): checking all headers, %q missing: %v",
+					spec.Procedure,
+					spec.IsClient,
+					key,
+					header,
+				),
+			)
+		}
+	}
+
+	// The client and handler interceptor onions are the meat of the test. The
+	// order of interceptor execution must be the same for unary and streaming
+	// procedures.
+	//
+	// Requests should fall through the client onion from top to bottom, traverse
+	// the network, and then fall through the handler onion from top to bottom.
+	// Responses should climb up the handler onion, traverse the network, and
+	// then climb up the client onion.
+	//
+	// The request and response sides of this onion are numbered to make the
+	// intended order clear.
+	clientOnion := triple.WithInterceptors(
+		newHeaderInterceptor(
+			// 1 (start). request: should see protocol-related headers
+			func(_ triple.Spec, h http.Header) {
+				assert.NotZero(t, h.Get("Content-Type"))
+			},
+			// 12 (end). response: check "one"-"four"
+			assertAllPresent,
+		),
+		newHeaderInterceptor(
+			newInspector("", "one"),       // 2. request: add header "one"
+			newInspector("three", "four"), // 11. response: check "three", add "four"
+		),
+		newHeaderInterceptor(
+			newInspector("one", "two"),   // 3. request: check "one", add "two"
+			newInspector("two", "three"), // 10. response: check "two", add "three"
+		),
+	)
+	handlerOnion := triple.WithInterceptors(
+		newHeaderInterceptor(
+			newInspector("two", "three"), // 4. request: check "two", add "three"
+			newInspector("one", "two"),   // 9. response: check "one", add "two"
+		),
+		newHeaderInterceptor(
+			newInspector("three", "four"), // 5. request: check "three", add "four"
+			newInspector("", "one"),       // 8. response: add "one"
+		),
+		newHeaderInterceptor(
+			assertAllPresent, // 6. request: check "one"-"four"
+			nil,              // 7. response: no-op
+		),
+	)
+
+	mux := http.NewServeMux()
+	mux.Handle(
+		pingv1connect.NewPingServiceHandler(
+			pingServer{},
+			handlerOnion,
+		),
+	)
+	server := httptest.NewServer(mux)
+	defer server.Close()
+
+	client := pingv1connect.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+		clientOnion,
+	)
+
+	err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{Number: 10}), triple.NewResponse(&pingv1.PingResponse{}))
+	assert.Nil(t, err)
+
+	//responses, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{Number: 10}))
+	//assert.Nil(t, err)
+	//var sum int64
+	//for responses.Receive(&pingv1.CountUpResponse{}) {
+	//	msg := responses.Msg().(pingv1.CountUpResponse)
+	//	sum += msg.Number
+	//}
+	//assert.Equal(t, sum, 55)
+	//assert.Nil(t, responses.Close())
+}
+
+//func TestEmptyUnaryInterceptorFunc(t *testing.T) {
+//	t.Parallel()
+//	mux := http.NewServeMux()
+//	interceptor := triple.UnaryInterceptorFunc(func(next triple.UnaryFunc) triple.UnaryFunc {
+//		return func(ctx context.Context, request triple.AnyRequest, response triple.AnyResponse) error {
+//			return next(ctx, request, response)
+//		}
+//	})
+//	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}, triple.WithInterceptors(interceptor)))
+//	server := httptest.NewServer(mux)
+//	t.Cleanup(server.Close)
+//	connectClient := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithInterceptors(interceptor))
+//	err := connectClient.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+//	assert.Nil(t, err)
+//	sumStream, err := connectClient.Sum(context.Background())
+//	assert.Nil(t, err)
+//	assert.Nil(t, sumStream.Send(&pingv1.SumRequest{Number: 1}))
+//	err = sumStream.CloseAndReceive(triple.NewResponse(&pingv1.SumResponse{}))
+//	assert.Nil(t, err)
+//	countUpStream, err := connectClient.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{}))
+//	assert.Nil(t, err)
+//	for countUpStream.Receive(&pingv1.CountUpResponse{}) {
+//		assert.NotNil(t, countUpStream.Msg())
+//	}
+//	assert.Nil(t, countUpStream.Close())
+//}
+
+// headerInterceptor makes it easier to write interceptors that inspect or
+// mutate HTTP headers. It applies the same logic to unary and streaming
+// procedures, wrapping the send or receive side of the stream as appropriate.
+//
+// It's useful as a testing harness to make sure that we're chaining
+// interceptors in the correct order.
+type headerInterceptor struct {
+	inspectRequestHeader  func(triple.Spec, http.Header)
+	inspectResponseHeader func(triple.Spec, http.Header)
+}
+
+// newHeaderInterceptor constructs a headerInterceptor. Nil function pointers
+// are treated as no-ops.
+func newHeaderInterceptor(
+	inspectRequestHeader func(triple.Spec, http.Header),
+	inspectResponseHeader func(triple.Spec, http.Header),
+) *headerInterceptor {
+	interceptor := headerInterceptor{
+		inspectRequestHeader:  inspectRequestHeader,
+		inspectResponseHeader: inspectResponseHeader,
+	}
+	if interceptor.inspectRequestHeader == nil {
+		interceptor.inspectRequestHeader = func(_ triple.Spec, _ http.Header) {}
+	}
+	if interceptor.inspectResponseHeader == nil {
+		interceptor.inspectResponseHeader = func(_ triple.Spec, _ http.Header) {}
+	}
+	return &interceptor
+}
+
+func (h *headerInterceptor) WrapUnaryHandler(next triple.UnaryHandlerFunc) triple.UnaryHandlerFunc {
+	return func(ctx context.Context, request triple.AnyRequest) (triple.AnyResponse, error) {
+		h.inspectRequestHeader(request.Spec(), request.Header())
+		response, err := next(ctx, request)
+		if err != nil {
+			return nil, err
+		}
+		h.inspectResponseHeader(request.Spec(), response.Header())
+		return response, nil
+	}
+}
+
+func (h *headerInterceptor) WrapUnary(next triple.UnaryFunc) triple.UnaryFunc {
+	return func(ctx context.Context, req triple.AnyRequest, res triple.AnyResponse) error {
+		h.inspectRequestHeader(req.Spec(), req.Header())
+		err := next(ctx, req, res)
+		if err != nil {
+			return err
+		}
+		h.inspectResponseHeader(req.Spec(), res.Header())
+		return nil
+	}
+}
+
+func (h *headerInterceptor) WrapStreamingClient(next triple.StreamingClientFunc) triple.StreamingClientFunc {
+	return func(ctx context.Context, spec triple.Spec) triple.StreamingClientConn {
+		return &headerInspectingClientConn{
+			StreamingClientConn:   next(ctx, spec),
+			inspectRequestHeader:  h.inspectRequestHeader,
+			inspectResponseHeader: h.inspectResponseHeader,
+		}
+	}
+}
+
+func (h *headerInterceptor) WrapStreamingHandler(next triple.StreamingHandlerFunc) triple.StreamingHandlerFunc {
+	return func(ctx context.Context, conn triple.StreamingHandlerConn) error {
+		h.inspectRequestHeader(conn.Spec(), conn.RequestHeader())
+		return next(ctx, &headerInspectingHandlerConn{
+			StreamingHandlerConn:  conn,
+			inspectResponseHeader: h.inspectResponseHeader,
+		})
+	}
+}
+
+type headerInspectingHandlerConn struct {
+	triple.StreamingHandlerConn
+
+	inspectedResponse     bool
+	inspectResponseHeader func(triple.Spec, http.Header)
+}
+
+func (hc *headerInspectingHandlerConn) Send(msg interface{}) error {
+	if !hc.inspectedResponse {
+		hc.inspectResponseHeader(hc.Spec(), hc.ResponseHeader())
+		hc.inspectedResponse = true
+	}
+	return hc.StreamingHandlerConn.Send(msg)
+}
+
+type headerInspectingClientConn struct {
+	triple.StreamingClientConn
+
+	inspectedRequest      bool
+	inspectRequestHeader  func(triple.Spec, http.Header)
+	inspectedResponse     bool
+	inspectResponseHeader func(triple.Spec, http.Header)
+}
+
+func (cc *headerInspectingClientConn) Send(msg interface{}) error {
+	if !cc.inspectedRequest {
+		cc.inspectRequestHeader(cc.Spec(), cc.RequestHeader())
+		cc.inspectedRequest = true
+	}
+	return cc.StreamingClientConn.Send(msg)
+}
+
+func (cc *headerInspectingClientConn) Receive(msg interface{}) error {
+	err := cc.StreamingClientConn.Receive(msg)
+	if !cc.inspectedResponse {
+		cc.inspectResponseHeader(cc.Spec(), cc.ResponseHeader())
+		cc.inspectedResponse = true
+	}
+	return err
+}
diff --git a/protocol/triple/triple_protocol/internal/assert/assert.go b/protocol/triple/triple_protocol/internal/assert/assert.go
new file mode 100644
index 0000000..c1226aa
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/assert/assert.go
@@ -0,0 +1,220 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF interface{} KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package assert is a minimal assert package using reflection.
+//
+// This prevents triple from needing additional dependencies.
+package assert
+
+import (
+	"bytes"
+	"errors"
+	"fmt"
+	"reflect"
+	"regexp"
+	"testing"
+)
+
+import (
+	"github.com/google/go-cmp/cmp"
+
+	"google.golang.org/protobuf/testing/protocmp"
+)
+
+// Equal asserts that two values are equal.
+func Equal(tb testing.TB, got, want interface{}, options ...Option) bool {
+	tb.Helper()
+	if cmpEqual(got, want) {
+		return true
+	}
+	report(tb, got, want, "assert.Equal", true /* showWant */, options...)
+	return false
+}
+
+// NotEqual asserts that two values are not equal.
+func NotEqual(tb testing.TB, got, want interface{}, options ...Option) bool {
+	tb.Helper()
+	if !cmpEqual(got, want) {
+		return true
+	}
+	report(tb, got, want, "assert.NotEqual", true /* showWant */, options...)
+	return false
+}
+
+// Nil asserts that the value is nil.
+func Nil(tb testing.TB, got interface{}, options ...Option) bool {
+	tb.Helper()
+	if isNil(got) {
+		return true
+	}
+	report(tb, got, nil, "assert.Nil", false /* showWant */, options...)
+	return false
+}
+
+// NotNil asserts that the value isn't nil.
+func NotNil(tb testing.TB, got interface{}, options ...Option) bool {
+	tb.Helper()
+	if !isNil(got) {
+		return true
+	}
+	report(tb, got, nil, "assert.NotNil", false /* showWant */, options...)
+	return false
+}
+
+// Zero asserts that the value is its type's zero value.
+func Zero(tb testing.TB, got interface{}, options ...Option) bool {
+	tb.Helper()
+	typ := reflect.TypeOf(got)
+	if typ == nil {
+		return true
+	}
+	want := reflect.Zero(typ).Interface()
+	if cmpEqual(got, want) {
+		return true
+	}
+	report(tb, got, want, fmt.Sprintf("assert.Zero (type %T)", got), false /* showWant */, options...)
+	return false
+}
+
+// NotZero asserts that the value is non-zero.
+func NotZero(tb testing.TB, got interface{}, options ...Option) bool {
+	tb.Helper()
+	typ := reflect.TypeOf(got)
+	if typ == nil {
+		return false
+	}
+	want := reflect.Zero(typ).Interface()
+	if !cmpEqual(got, want) {
+		return true
+	}
+	report(tb, got, want, fmt.Sprintf("assert.NotZero (type %T)", got), false /* showWant */, options...)
+	return false
+}
+
+// Match asserts that the value matches a regexp.
+func Match(tb testing.TB, got, want string, options ...Option) bool {
+	tb.Helper()
+	re, err := regexp.Compile(want)
+	if err != nil {
+		tb.Fatalf("invalid regexp %q: %v", want, err)
+	}
+	if re.MatchString(got) {
+		return true
+	}
+	report(tb, got, want, "assert.Match", true /* showWant */, options...)
+	return false
+}
+
+// ErrorIs asserts that "want" is in "got's" error chain. See the standard
+// library's errors package for details on error chains. On failure, output is
+// identical to Equal.
+func ErrorIs(tb testing.TB, got, want error, options ...Option) bool {
+	tb.Helper()
+	if errors.Is(got, want) {
+		return true
+	}
+	report(tb, got, want, "assert.ErrorIs", true /* showWant */, options...)
+	return false
+}
+
+// False asserts that "got" is false.
+func False(tb testing.TB, got bool, options ...Option) bool {
+	tb.Helper()
+	if !got {
+		return true
+	}
+	report(tb, got, false, "assert.False", false /* showWant */, options...)
+	return false
+}
+
+// True asserts that "got" is true.
+func True(tb testing.TB, got bool, options ...Option) bool {
+	tb.Helper()
+	if got {
+		return true
+	}
+	report(tb, got, true, "assert.True", false /* showWant */, options...)
+	return false
+}
+
+// Panics asserts that the function called panics.
+func Panics(tb testing.TB, panicker func(), options ...Option) {
+	tb.Helper()
+	defer func() {
+		if r := recover(); r == nil {
+			report(tb, r, nil, "assert.Panic", false /* showWant */, options...)
+		}
+	}()
+	panicker()
+}
+
+// An Option configures an assertion.
+type Option interface {
+	// Only option we've needed so far is a formatted message, so we can keep
+	// this simple.
+	message() string
+}
+
+// Sprintf adds a user-defined message to the assertion's output. The arguments
+// are passed directly to fmt.Sprintf for formatting.
+//
+// If Sprintf is passed multiple times, only the last message is used.
+func Sprintf(template string, args ...interface{}) Option {
+	return &sprintfOption{fmt.Sprintf(template, args...)}
+}
+
+type sprintfOption struct {
+	msg string
+}
+
+func (o *sprintfOption) message() string {
+	return o.msg
+}
+
+func report(tb testing.TB, got, want interface{}, desc string, showWant bool, options ...Option) {
+	tb.Helper()
+	buffer := &bytes.Buffer{}
+	if len(options) > 0 {
+		buffer.WriteString(options[len(options)-1].message())
+	}
+	buffer.WriteString("\n")
+	fmt.Fprintf(buffer, "assertion:\t%s\n", desc)
+	fmt.Fprintf(buffer, "got:\t%+v\n", got)
+	if showWant {
+		fmt.Fprintf(buffer, "want:\t%+v\n", want)
+	}
+	tb.Fatal(buffer.String())
+}
+
+func isNil(got interface{}) bool {
+	// Simple case, true only when the user directly passes a literal nil.
+	if got == nil {
+		return true
+	}
+	// Possibly more complex. Interfaces are a pair of words: a pointer to a type
+	// and a pointer to a value. Because we're passing got as an interface, it's
+	// likely that we've gotten a non-nil type and a nil value. This makes got
+	// itself non-nil, but the user's code passed a nil value.
+	val := reflect.ValueOf(got)
+	switch val.Kind() {
+	case reflect.Chan, reflect.Func, reflect.Interface, reflect.Map, reflect.Ptr, reflect.Slice:
+		return val.IsNil()
+	default:
+		return false
+	}
+}
+
+func cmpEqual(got, want interface{}) bool {
+	return cmp.Equal(got, want, protocmp.Transform())
+}
diff --git a/protocol/triple/triple_protocol/internal/assert/assert_test.go b/protocol/triple/triple_protocol/internal/assert/assert_test.go
new file mode 100644
index 0000000..79d55f1
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/assert/assert_test.go
@@ -0,0 +1,98 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package assert
+
+import (
+	"errors"
+	"fmt"
+	"testing"
+)
+
+func TestAssertions(t *testing.T) {
+	t.Parallel()
+
+	t.Run("equal", func(t *testing.T) {
+		t.Parallel()
+		Equal(t, 1, 1, Sprintf("1 == %d", 1))
+		NotEqual(t, 1, 2)
+	})
+
+	t.Run("nil", func(t *testing.T) {
+		t.Parallel()
+		Nil(t, nil)
+		Nil(t, (*chan int)(nil))
+		Nil(t, (*func())(nil))
+		Nil(t, (*map[int]int)(nil))
+		Nil(t, (*pair)(nil))
+		Nil(t, (*[]int)(nil))
+
+		NotNil(t, make(chan int))
+		NotNil(t, func() {})
+		NotNil(t, interface{}(1))
+		NotNil(t, make(map[int]int))
+		NotNil(t, &pair{})
+		NotNil(t, make([]int, 0))
+
+		NotNil(t, "foo")
+		NotNil(t, 0)
+		NotNil(t, false)
+		NotNil(t, pair{})
+	})
+
+	t.Run("zero", func(t *testing.T) {
+		t.Parallel()
+		var n *int
+		Zero(t, n)
+		var p pair
+		Zero(t, p)
+		var null *pair
+		Zero(t, null)
+		var s []int
+		Zero(t, s)
+		var m map[string]string
+		Zero(t, m)
+		var nilIntf interface{}
+		Zero(t, nilIntf)
+		NotZero(t, nilIntf)
+	})
+
+	t.Run("error chain", func(t *testing.T) {
+		t.Parallel()
+		want := errors.New("base error")
+		ErrorIs(t, fmt.Errorf("context: %w", want), want)
+	})
+
+	t.Run("exported fields", func(t *testing.T) {
+		t.Parallel()
+		// NotEqual can only handle exported fields.
+		p1 := pair{1, 2}
+		p2 := pair{1, 3}
+		NotEqual(t, p1, p2)
+	})
+
+	t.Run("regexp", func(t *testing.T) {
+		t.Parallel()
+		Match(t, "foobar", `^foo`)
+	})
+
+	t.Run("panics", func(t *testing.T) {
+		t.Parallel()
+		Panics(t, func() { panic("testing") }) //nolint:forbidigo
+	})
+}
+
+type pair struct {
+	First, Second int
+}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connect/collide/v1/collide.pb.go b/protocol/triple/triple_protocol/internal/gen/proto/connect/collide/v1/collide.pb.go
new file mode 100644
index 0000000..64deaa0
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connect/collide/v1/collide.pb.go
@@ -0,0 +1,227 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        (unknown)
+// source: proto/triple/collide/v1/collide.proto
+
+package collidev1
+
+import (
+	reflect "reflect"
+	sync "sync"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type ImportRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *ImportRequest) Reset() {
+	*x = ImportRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_collide_v1_collide_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ImportRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ImportRequest) ProtoMessage() {}
+
+func (x *ImportRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_collide_v1_collide_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ImportRequest.ProtoReflect.Descriptor instead.
+func (*ImportRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_collide_v1_collide_proto_rawDescGZIP(), []int{0}
+}
+
+type ImportResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *ImportResponse) Reset() {
+	*x = ImportResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_collide_v1_collide_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *ImportResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*ImportResponse) ProtoMessage() {}
+
+func (x *ImportResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_collide_v1_collide_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use ImportResponse.ProtoReflect.Descriptor instead.
+func (*ImportResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_collide_v1_collide_proto_rawDescGZIP(), []int{1}
+}
+
+var File_proto_connect_collide_v1_collide_proto protoreflect.FileDescriptor
+
+var file_proto_connect_collide_v1_collide_proto_rawDesc = []byte{
+	0x0a, 0x26, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,
+	0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2f, 0x76, 0x31, 0x2f, 0x63, 0x6f, 0x6c, 0x6c, 0x69,
+	0x64, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x12, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,
+	0x74, 0x2e, 0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2e, 0x76, 0x31, 0x22, 0x0f, 0x0a, 0x0d,
+	0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x10, 0x0a,
+	0x0e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x32,
+	0x63, 0x0a, 0x0e, 0x43, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63,
+	0x65, 0x12, 0x51, 0x0a, 0x06, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x12, 0x21, 0x2e, 0x63, 0x6f,
+	0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2e, 0x76, 0x31,
+	0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x22,
+	0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65,
+	0x2e, 0x76, 0x31, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x22, 0x00, 0x42, 0xf3, 0x01, 0x0a, 0x16, 0x63, 0x6f, 0x6d, 0x2e, 0x63, 0x6f, 0x6e,
+	0x6e, 0x65, 0x63, 0x74, 0x2e, 0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2e, 0x76, 0x31, 0x42,
+	0x0c, 0x43, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a,
+	0x61, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63,
+	0x68, 0x65, 0x2f, 0x64, 0x75, 0x62, 0x62, 0x6f, 0x2d, 0x67, 0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x2f, 0x74, 0x72, 0x69, 0x70,
+	0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x63, 0x6f,
+	0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2f, 0x76, 0x31, 0x3b, 0x63, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65,
+	0x76, 0x31, 0xa2, 0x02, 0x03, 0x43, 0x43, 0x58, 0xaa, 0x02, 0x12, 0x43, 0x6f, 0x6e, 0x6e, 0x65,
+	0x63, 0x74, 0x2e, 0x43, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x12,
+	0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5c, 0x43, 0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x5c,
+	0x56, 0x31, 0xe2, 0x02, 0x1e, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5c, 0x43, 0x6f, 0x6c,
+	0x6c, 0x69, 0x64, 0x65, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64,
+	0x61, 0x74, 0x61, 0xea, 0x02, 0x14, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x3a, 0x3a, 0x43,
+	0x6f, 0x6c, 0x6c, 0x69, 0x64, 0x65, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x33,
+}
+
+var (
+	file_proto_connect_collide_v1_collide_proto_rawDescOnce sync.Once
+	file_proto_connect_collide_v1_collide_proto_rawDescData = file_proto_connect_collide_v1_collide_proto_rawDesc
+)
+
+func file_proto_connect_collide_v1_collide_proto_rawDescGZIP() []byte {
+	file_proto_connect_collide_v1_collide_proto_rawDescOnce.Do(func() {
+		file_proto_connect_collide_v1_collide_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_connect_collide_v1_collide_proto_rawDescData)
+	})
+	return file_proto_connect_collide_v1_collide_proto_rawDescData
+}
+
+var file_proto_connect_collide_v1_collide_proto_msgTypes = make([]protoimpl.MessageInfo, 2)
+var file_proto_connect_collide_v1_collide_proto_goTypes = []interface{}{
+	(*ImportRequest)(nil),  // 0: triple.collide.v1.ImportRequest
+	(*ImportResponse)(nil), // 1: triple.collide.v1.ImportResponse
+}
+var file_proto_connect_collide_v1_collide_proto_depIdxs = []int32{
+	0, // 0: triple.collide.v1.CollideService.Import:input_type -> triple.collide.v1.ImportRequest
+	1, // 1: triple.collide.v1.CollideService.Import:output_type -> triple.collide.v1.ImportResponse
+	1, // [1:2] is the sub-list for method output_type
+	0, // [0:1] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_proto_connect_collide_v1_collide_proto_init() }
+func file_proto_connect_collide_v1_collide_proto_init() {
+	if File_proto_connect_collide_v1_collide_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_proto_connect_collide_v1_collide_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ImportRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_collide_v1_collide_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*ImportResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_proto_connect_collide_v1_collide_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   2,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_proto_connect_collide_v1_collide_proto_goTypes,
+		DependencyIndexes: file_proto_connect_collide_v1_collide_proto_depIdxs,
+		MessageInfos:      file_proto_connect_collide_v1_collide_proto_msgTypes,
+	}.Build()
+	File_proto_connect_collide_v1_collide_proto = out.File
+	file_proto_connect_collide_v1_collide_proto_rawDesc = nil
+	file_proto_connect_collide_v1_collide_proto_goTypes = nil
+	file_proto_connect_collide_v1_collide_proto_depIdxs = nil
+}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/import.pb.go b/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/import.pb.go
new file mode 100644
index 0000000..2483976
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/import.pb.go
@@ -0,0 +1,96 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        (unknown)
+// source: proto/triple/import/v1/import.proto
+
+package importv1
+
+import (
+	reflect "reflect"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+var File_proto_connect_import_v1_import_proto protoreflect.FileDescriptor
+
+var file_proto_connect_import_v1_import_proto_rawDesc = []byte{
+	0x0a, 0x24, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,
+	0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x2f, 0x76, 0x31, 0x2f, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74,
+	0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x11, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e,
+	0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x76, 0x31, 0x32, 0x0f, 0x0a, 0x0d, 0x49, 0x6d, 0x70,
+	0x6f, 0x72, 0x74, 0x53, 0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x42, 0xeb, 0x01, 0x0a, 0x15, 0x63,
+	0x6f, 0x6d, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x69, 0x6d, 0x70, 0x6f, 0x72,
+	0x74, 0x2e, 0x76, 0x31, 0x42, 0x0b, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x50, 0x72, 0x6f, 0x74,
+	0x6f, 0x50, 0x01, 0x5a, 0x5f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f,
+	0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x64, 0x75, 0x62, 0x62, 0x6f, 0x2d, 0x67, 0x6f, 0x2f,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x2f,
+	0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f,
+	0x67, 0x65, 0x6e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63,
+	0x74, 0x2f, 0x69, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x2f, 0x76, 0x31, 0x3b, 0x69, 0x6d, 0x70, 0x6f,
+	0x72, 0x74, 0x76, 0x31, 0xa2, 0x02, 0x03, 0x43, 0x49, 0x58, 0xaa, 0x02, 0x11, 0x43, 0x6f, 0x6e,
+	0x6e, 0x65, 0x63, 0x74, 0x2e, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x2e, 0x56, 0x31, 0xca, 0x02,
+	0x11, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5c, 0x49, 0x6d, 0x70, 0x6f, 0x72, 0x74, 0x5c,
+	0x56, 0x31, 0xe2, 0x02, 0x1d, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5c, 0x49, 0x6d, 0x70,
+	0x6f, 0x72, 0x74, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61,
+	0x74, 0x61, 0xea, 0x02, 0x13, 0x43, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x3a, 0x3a, 0x49, 0x6d,
+	0x70, 0x6f, 0x72, 0x74, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var file_proto_connect_import_v1_import_proto_goTypes = []interface{}{}
+var file_proto_connect_import_v1_import_proto_depIdxs = []int32{
+	0, // [0:0] is the sub-list for method output_type
+	0, // [0:0] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_proto_connect_import_v1_import_proto_init() }
+func file_proto_connect_import_v1_import_proto_init() {
+	if File_proto_connect_import_v1_import_proto != nil {
+		return
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_proto_connect_import_v1_import_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   0,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_proto_connect_import_v1_import_proto_goTypes,
+		DependencyIndexes: file_proto_connect_import_v1_import_proto_depIdxs,
+	}.Build()
+	File_proto_connect_import_v1_import_proto = out.File
+	file_proto_connect_import_v1_import_proto_rawDesc = nil
+	file_proto_connect_import_v1_import_proto_goTypes = nil
+	file_proto_connect_import_v1_import_proto_depIdxs = nil
+}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/importv1connect/import.connect.go b/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/importv1connect/import.connect.go
new file mode 100644
index 0000000..63785f7
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/importv1connect/import.connect.go
@@ -0,0 +1,80 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-connect-go. DO NOT EDIT.
+//
+// Source: connect/import/v1/import.proto
+
+package importv1connect
+
+import (
+	http "net/http"
+	strings "strings"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	_ "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1"
+)
+
+// This is a compile-time assertion to ensure that this generated file and the connect package are
+// compatible. If you get a compiler error that this constant is not defined, this code was
+// generated with a version of connect newer than the one compiled into your binary. You can fix the
+// problem by either regenerating this code with an older version of connect or updating the connect
+// version compiled into your binary.
+const (
+	// ImportServiceName is the fully-qualified name of the ImportService service.
+	ImportServiceName = "connect.import.v1.ImportService"
+)
+
+// ImportServiceClient is a client for the connect.import.v1.ImportService service.
+type ImportServiceClient interface {
+}
+
+// NewImportServiceClient constructs a client for the connect.import.v1.ImportService service. By
+// default, it uses the Connect protocol with the binary Protobuf Codec, asks for gzipped responses,
+// and sends uncompressed requests. To use the gRPC or gRPC-Web protocols, supply the
+// connect.WithGRPC() or connect.WithGRPCWeb() options.
+//
+// The URL supplied here should be the base URL for the Connect or gRPC server (for example,
+// http://api.acme.com or https://acme.com/grpc).
+func NewImportServiceClient(httpClient triple_protocol.HTTPClient, baseURL string, opts ...triple_protocol.ClientOption) ImportServiceClient {
+	baseURL = strings.TrimRight(baseURL, "/")
+	return &importServiceClient{}
+}
+
+// importServiceClient implements ImportServiceClient.
+type importServiceClient struct {
+}
+
+// ImportServiceHandler is an implementation of the connect.import.v1.ImportService service.
+type ImportServiceHandler interface {
+}
+
+// NewImportServiceHandler builds an HTTP handler from the service implementation. It returns the
+// path on which to mount the handler and the handler itself.
+//
+// By default, handlers support the Connect, gRPC, and gRPC-Web protocols with the binary Protobuf
+// and JSON codecs. They also support gzip compression.
+func NewImportServiceHandler(svc ImportServiceHandler, opts ...triple_protocol.HandlerOption) (string, http.Handler) {
+	return "/connect.import.v1.ImportService/", http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		switch r.URL.Path {
+		default:
+			http.NotFound(w, r)
+		}
+	})
+}
+
+// UnimplementedImportServiceHandler returns CodeUnimplemented from all methods.
+type UnimplementedImportServiceHandler struct{}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/ping.pb.go b/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/ping.pb.go
new file mode 100644
index 0000000..5a13a86
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/ping.pb.go
@@ -0,0 +1,785 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// The canonical location for this file is
+// https://github.com/bufbuild/connect-go/blob/main/internal/proto/connect/ping/v1/ping.proto.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        (unknown)
+// source: proto/triple/ping/v1/ping.proto
+
+// The triple.ping.v1 package contains an echo service designed to test the
+// triple-go implementation.
+
+package pingv1
+
+import (
+	reflect "reflect"
+	sync "sync"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+type PingRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64  `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+	Text   string `protobuf:"bytes,2,opt,name=text,proto3" json:"text,omitempty"`
+}
+
+func (x *PingRequest) Reset() {
+	*x = PingRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PingRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PingRequest) ProtoMessage() {}
+
+func (x *PingRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PingRequest.ProtoReflect.Descriptor instead.
+func (*PingRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *PingRequest) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+func (x *PingRequest) GetText() string {
+	if x != nil {
+		return x.Text
+	}
+	return ""
+}
+
+type PingResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64  `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+	Text   string `protobuf:"bytes,2,opt,name=text,proto3" json:"text,omitempty"`
+}
+
+func (x *PingResponse) Reset() {
+	*x = PingResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[1]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *PingResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*PingResponse) ProtoMessage() {}
+
+func (x *PingResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[1]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use PingResponse.ProtoReflect.Descriptor instead.
+func (*PingResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{1}
+}
+
+func (x *PingResponse) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+func (x *PingResponse) GetText() string {
+	if x != nil {
+		return x.Text
+	}
+	return ""
+}
+
+type FailRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Code int32 `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"`
+}
+
+func (x *FailRequest) Reset() {
+	*x = FailRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[2]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FailRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FailRequest) ProtoMessage() {}
+
+func (x *FailRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[2]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FailRequest.ProtoReflect.Descriptor instead.
+func (*FailRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{2}
+}
+
+func (x *FailRequest) GetCode() int32 {
+	if x != nil {
+		return x.Code
+	}
+	return 0
+}
+
+type FailResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+}
+
+func (x *FailResponse) Reset() {
+	*x = FailResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[3]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *FailResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*FailResponse) ProtoMessage() {}
+
+func (x *FailResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[3]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use FailResponse.ProtoReflect.Descriptor instead.
+func (*FailResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{3}
+}
+
+type SumRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+}
+
+func (x *SumRequest) Reset() {
+	*x = SumRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[4]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SumRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SumRequest) ProtoMessage() {}
+
+func (x *SumRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[4]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SumRequest.ProtoReflect.Descriptor instead.
+func (*SumRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{4}
+}
+
+func (x *SumRequest) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+type SumResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Sum int64 `protobuf:"varint,1,opt,name=sum,proto3" json:"sum,omitempty"`
+}
+
+func (x *SumResponse) Reset() {
+	*x = SumResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[5]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *SumResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*SumResponse) ProtoMessage() {}
+
+func (x *SumResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[5]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use SumResponse.ProtoReflect.Descriptor instead.
+func (*SumResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{5}
+}
+
+func (x *SumResponse) GetSum() int64 {
+	if x != nil {
+		return x.Sum
+	}
+	return 0
+}
+
+type CountUpRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+}
+
+func (x *CountUpRequest) Reset() {
+	*x = CountUpRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[6]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CountUpRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CountUpRequest) ProtoMessage() {}
+
+func (x *CountUpRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[6]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CountUpRequest.ProtoReflect.Descriptor instead.
+func (*CountUpRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{6}
+}
+
+func (x *CountUpRequest) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+type CountUpResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+}
+
+func (x *CountUpResponse) Reset() {
+	*x = CountUpResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[7]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CountUpResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CountUpResponse) ProtoMessage() {}
+
+func (x *CountUpResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[7]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CountUpResponse.ProtoReflect.Descriptor instead.
+func (*CountUpResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{7}
+}
+
+func (x *CountUpResponse) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+type CumSumRequest struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Number int64 `protobuf:"varint,1,opt,name=number,proto3" json:"number,omitempty"`
+}
+
+func (x *CumSumRequest) Reset() {
+	*x = CumSumRequest{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[8]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CumSumRequest) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CumSumRequest) ProtoMessage() {}
+
+func (x *CumSumRequest) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[8]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CumSumRequest.ProtoReflect.Descriptor instead.
+func (*CumSumRequest) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{8}
+}
+
+func (x *CumSumRequest) GetNumber() int64 {
+	if x != nil {
+		return x.Number
+	}
+	return 0
+}
+
+type CumSumResponse struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Sum int64 `protobuf:"varint,1,opt,name=sum,proto3" json:"sum,omitempty"`
+}
+
+func (x *CumSumResponse) Reset() {
+	*x = CumSumResponse{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[9]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *CumSumResponse) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*CumSumResponse) ProtoMessage() {}
+
+func (x *CumSumResponse) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connect_ping_v1_ping_proto_msgTypes[9]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use CumSumResponse.ProtoReflect.Descriptor instead.
+func (*CumSumResponse) Descriptor() ([]byte, []int) {
+	return file_proto_connect_ping_v1_ping_proto_rawDescGZIP(), []int{9}
+}
+
+func (x *CumSumResponse) GetSum() int64 {
+	if x != nil {
+		return x.Sum
+	}
+	return 0
+}
+
+var File_proto_connect_ping_v1_ping_proto protoreflect.FileDescriptor
+
+var file_proto_connect_ping_v1_ping_proto_rawDesc = []byte{
+	0x0a, 0x20, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f,
+	0x70, 0x69, 0x6e, 0x67, 0x2f, 0x76, 0x31, 0x2f, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x70, 0x72, 0x6f,
+	0x74, 0x6f, 0x12, 0x0f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67,
+	0x2e, 0x76, 0x31, 0x22, 0x39, 0x0a, 0x0b, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65,
+	0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01,
+	0x28, 0x03, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65,
+	0x78, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x65, 0x78, 0x74, 0x22, 0x3a,
+	0x0a, 0x0c, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x16,
+	0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06,
+	0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x74, 0x65, 0x78, 0x74, 0x18, 0x02,
+	0x20, 0x01, 0x28, 0x09, 0x52, 0x04, 0x74, 0x65, 0x78, 0x74, 0x22, 0x21, 0x0a, 0x0b, 0x46, 0x61,
+	0x69, 0x6c, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64,
+	0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x22, 0x0e, 0x0a,
+	0x0c, 0x46, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x24, 0x0a,
+	0x0a, 0x53, 0x75, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e,
+	0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x75, 0x6d,
+	0x62, 0x65, 0x72, 0x22, 0x1f, 0x0a, 0x0b, 0x53, 0x75, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e,
+	0x73, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52,
+	0x03, 0x73, 0x75, 0x6d, 0x22, 0x28, 0x0a, 0x0e, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x70, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72,
+	0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x29,
+	0x0a, 0x0f, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73,
+	0x65, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x03, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62, 0x65, 0x72, 0x22, 0x27, 0x0a, 0x0d, 0x43, 0x75, 0x6d,
+	0x53, 0x75, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x16, 0x0a, 0x06, 0x6e, 0x75,
+	0x6d, 0x62, 0x65, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x6e, 0x75, 0x6d, 0x62,
+	0x65, 0x72, 0x22, 0x22, 0x0a, 0x0e, 0x43, 0x75, 0x6d, 0x53, 0x75, 0x6d, 0x52, 0x65, 0x73, 0x70,
+	0x6f, 0x6e, 0x73, 0x65, 0x12, 0x10, 0x0a, 0x03, 0x73, 0x75, 0x6d, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x03, 0x52, 0x03, 0x73, 0x75, 0x6d, 0x32, 0x87, 0x03, 0x0a, 0x0b, 0x50, 0x69, 0x6e, 0x67, 0x53,
+	0x65, 0x72, 0x76, 0x69, 0x63, 0x65, 0x12, 0x48, 0x0a, 0x04, 0x50, 0x69, 0x6e, 0x67, 0x12, 0x1c,
+	0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31,
+	0x2e, 0x50, 0x69, 0x6e, 0x67, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x63,
+	0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x50,
+	0x69, 0x6e, 0x67, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x03, 0x90, 0x02, 0x01,
+	0x12, 0x45, 0x0a, 0x04, 0x46, 0x61, 0x69, 0x6c, 0x12, 0x1c, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,
+	0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x52,
+	0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1d, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,
+	0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x46, 0x61, 0x69, 0x6c, 0x52, 0x65, 0x73,
+	0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x12, 0x44, 0x0a, 0x03, 0x53, 0x75, 0x6d, 0x12, 0x1b,
+	0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31,
+	0x2e, 0x53, 0x75, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1c, 0x2e, 0x63, 0x6f,
+	0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x75,
+	0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x12, 0x50, 0x0a,
+	0x07, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x55, 0x70, 0x12, 0x1f, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65,
+	0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x75, 0x6e, 0x74,
+	0x55, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x20, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,
+	0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x6f, 0x75, 0x6e,
+	0x74, 0x55, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x30, 0x01, 0x12,
+	0x4f, 0x0a, 0x06, 0x43, 0x75, 0x6d, 0x53, 0x75, 0x6d, 0x12, 0x1e, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,
+	0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x75, 0x6d, 0x53,
+	0x75, 0x6d, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x1a, 0x1f, 0x2e, 0x63, 0x6f, 0x6e, 0x6e,
+	0x65, 0x63, 0x74, 0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x75, 0x6d, 0x53,
+	0x75, 0x6d, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x00, 0x28, 0x01, 0x30, 0x01,
+	0x42, 0xdb, 0x01, 0x0a, 0x13, 0x63, 0x6f, 0x6d, 0x2e, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74,
+	0x2e, 0x70, 0x69, 0x6e, 0x67, 0x2e, 0x76, 0x31, 0x42, 0x09, 0x50, 0x69, 0x6e, 0x67, 0x50, 0x72,
+	0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x5b, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f,
+	0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x64, 0x75, 0x62, 0x62, 0x6f, 0x2d, 0x67,
+	0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c,
+	0x65, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f,
+	0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,
+	0x65, 0x63, 0x74, 0x2f, 0x70, 0x69, 0x6e, 0x67, 0x2f, 0x76, 0x31, 0x3b, 0x70, 0x69, 0x6e, 0x67,
+	0x76, 0x31, 0xa2, 0x02, 0x03, 0x43, 0x50, 0x58, 0xaa, 0x02, 0x0f, 0x43, 0x6f, 0x6e, 0x6e, 0x65,
+	0x63, 0x74, 0x2e, 0x50, 0x69, 0x6e, 0x67, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x0f, 0x43, 0x6f, 0x6e,
+	0x6e, 0x65, 0x63, 0x74, 0x5c, 0x50, 0x69, 0x6e, 0x67, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x1b, 0x43,
+	0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x5c, 0x50, 0x69, 0x6e, 0x67, 0x5c, 0x56, 0x31, 0x5c, 0x47,
+	0x50, 0x42, 0x4d, 0x65, 0x74, 0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x11, 0x43, 0x6f, 0x6e,
+	0x6e, 0x65, 0x63, 0x74, 0x3a, 0x3a, 0x50, 0x69, 0x6e, 0x67, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_proto_connect_ping_v1_ping_proto_rawDescOnce sync.Once
+	file_proto_connect_ping_v1_ping_proto_rawDescData = file_proto_connect_ping_v1_ping_proto_rawDesc
+)
+
+func file_proto_connect_ping_v1_ping_proto_rawDescGZIP() []byte {
+	file_proto_connect_ping_v1_ping_proto_rawDescOnce.Do(func() {
+		file_proto_connect_ping_v1_ping_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_connect_ping_v1_ping_proto_rawDescData)
+	})
+	return file_proto_connect_ping_v1_ping_proto_rawDescData
+}
+
+var file_proto_connect_ping_v1_ping_proto_msgTypes = make([]protoimpl.MessageInfo, 10)
+var file_proto_connect_ping_v1_ping_proto_goTypes = []interface{}{
+	(*PingRequest)(nil),     // 0: triple.ping.v1.PingRequest
+	(*PingResponse)(nil),    // 1: triple.ping.v1.PingResponse
+	(*FailRequest)(nil),     // 2: triple.ping.v1.FailRequest
+	(*FailResponse)(nil),    // 3: triple.ping.v1.FailResponse
+	(*SumRequest)(nil),      // 4: triple.ping.v1.SumRequest
+	(*SumResponse)(nil),     // 5: triple.ping.v1.SumResponse
+	(*CountUpRequest)(nil),  // 6: triple.ping.v1.CountUpRequest
+	(*CountUpResponse)(nil), // 7: triple.ping.v1.CountUpResponse
+	(*CumSumRequest)(nil),   // 8: triple.ping.v1.CumSumRequest
+	(*CumSumResponse)(nil),  // 9: triple.ping.v1.CumSumResponse
+}
+var file_proto_connect_ping_v1_ping_proto_depIdxs = []int32{
+	0, // 0: triple.ping.v1.PingService.Ping:input_type -> triple.ping.v1.PingRequest
+	2, // 1: triple.ping.v1.PingService.Fail:input_type -> triple.ping.v1.FailRequest
+	4, // 2: triple.ping.v1.PingService.Sum:input_type -> triple.ping.v1.SumRequest
+	6, // 3: triple.ping.v1.PingService.CountUp:input_type -> triple.ping.v1.CountUpRequest
+	8, // 4: triple.ping.v1.PingService.CumSum:input_type -> triple.ping.v1.CumSumRequest
+	1, // 5: triple.ping.v1.PingService.Ping:output_type -> triple.ping.v1.PingResponse
+	3, // 6: triple.ping.v1.PingService.Fail:output_type -> triple.ping.v1.FailResponse
+	5, // 7: triple.ping.v1.PingService.Sum:output_type -> triple.ping.v1.SumResponse
+	7, // 8: triple.ping.v1.PingService.CountUp:output_type -> triple.ping.v1.CountUpResponse
+	9, // 9: triple.ping.v1.PingService.CumSum:output_type -> triple.ping.v1.CumSumResponse
+	5, // [5:10] is the sub-list for method output_type
+	0, // [0:5] is the sub-list for method input_type
+	0, // [0:0] is the sub-list for extension type_name
+	0, // [0:0] is the sub-list for extension extendee
+	0, // [0:0] is the sub-list for field type_name
+}
+
+func init() { file_proto_connect_ping_v1_ping_proto_init() }
+func file_proto_connect_ping_v1_ping_proto_init() {
+	if File_proto_connect_ping_v1_ping_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_proto_connect_ping_v1_ping_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PingRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[1].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*PingResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[2].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FailRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[3].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*FailResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SumRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*SumResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CountUpRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CountUpResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[8].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CumSumRequest); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+		file_proto_connect_ping_v1_ping_proto_msgTypes[9].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*CumSumResponse); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_proto_connect_ping_v1_ping_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   10,
+			NumExtensions: 0,
+			NumServices:   1,
+		},
+		GoTypes:           file_proto_connect_ping_v1_ping_proto_goTypes,
+		DependencyIndexes: file_proto_connect_ping_v1_ping_proto_depIdxs,
+		MessageInfos:      file_proto_connect_ping_v1_ping_proto_msgTypes,
+	}.Build()
+	File_proto_connect_ping_v1_ping_proto = out.File
+	file_proto_connect_ping_v1_ping_proto_rawDesc = nil
+	file_proto_connect_ping_v1_ping_proto_goTypes = nil
+	file_proto_connect_ping_v1_ping_proto_depIdxs = nil
+}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect/ping.connect.go b/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect/ping.connect.go
new file mode 100644
index 0000000..07bf7b3
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect/ping.connect.go
@@ -0,0 +1,243 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// The canonical location for this file is
+// https://github.com/bufbuild/connect-go/blob/main/internal/proto/connect/ping/v1/ping.proto.
+
+// Code generated by protoc-gen-connect-go. DO NOT EDIT.
+//
+// Source: connect/ping/v1/ping.proto
+
+// The connect.ping.v1 package contains an echo service designed to test the
+// connect-go implementation.
+package pingv1connect
+
+import (
+	context "context"
+	errors "errors"
+	http "net/http"
+	strings "strings"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+)
+
+const (
+	// PingServiceName is the fully-qualified name of the PingService service.
+	PingServiceName = "connect.ping.v1.PingService"
+)
+
+// These constants are the fully-qualified names of the RPCs defined in this package. They're
+// exposed at runtime as Spec.Procedure and as the final two segments of the HTTP route.
+//
+// Note that these are different from the fully-qualified method names used by
+// google.golang.org/protobuf/reflect/protoreflect. To convert from these constants to
+// reflection-formatted method names, remove the leading slash and convert the remaining slash to a
+// period.
+const (
+	// PingServicePingProcedure is the fully-qualified name of the PingService's Ping RPC.
+	PingServicePingProcedure = "/connect.ping.v1.PingService/Ping"
+	// PingServiceFailProcedure is the fully-qualified name of the PingService's Fail RPC.
+	PingServiceFailProcedure = "/connect.ping.v1.PingService/Fail"
+	// PingServiceSumProcedure is the fully-qualified name of the PingService's Sum RPC.
+	PingServiceSumProcedure = "/connect.ping.v1.PingService/Sum"
+	// PingServiceCountUpProcedure is the fully-qualified name of the PingService's CountUp RPC.
+	PingServiceCountUpProcedure = "/connect.ping.v1.PingService/CountUp"
+	// PingServiceCumSumProcedure is the fully-qualified name of the PingService's CumSum RPC.
+	PingServiceCumSumProcedure = "/connect.ping.v1.PingService/CumSum"
+)
+
+// PingServiceClient is a client for the connect.ping.v1.PingService service.
+type PingServiceClient interface {
+	// Ping sends a ping to the server to determine if it's reachable.
+	Ping(context.Context, *triple_protocol.Request, *triple_protocol.Response) error
+	// Fail always fails.
+	Fail(context.Context, *triple_protocol.Request, *triple_protocol.Response) error
+	// Sum calculates the sum of the numbers sent on the stream.
+	Sum(context.Context) (*triple_protocol.ClientStreamForClient,error)
+	// CountUp returns a stream of the numbers up to the given request.
+	CountUp(context.Context, *triple_protocol.Request) (*triple_protocol.ServerStreamForClient, error)
+	// CumSum determines the cumulative sum of all the numbers sent on the stream.
+	CumSum(context.Context) (*triple_protocol.BidiStreamForClient,error)
+}
+
+// NewPingServiceClient constructs a client for the connect.ping.v1.PingService service. By default,
+// it uses the Connect protocol with the binary Protobuf Codec, asks for gzipped responses, and
+// sends uncompressed requests. To use the gRPC or gRPC-Web protocols, supply the connect.WithGRPC()
+// or connect.WithGRPCWeb() options.
+//
+// The URL supplied here should be the base URL for the Connect or gRPC server (for example,
+// http://api.acme.com or https://acme.com/grpc).
+func NewPingServiceClient(httpClient triple_protocol.HTTPClient, baseURL string, opts ...triple_protocol.ClientOption) PingServiceClient {
+	baseURL = strings.TrimRight(baseURL, "/")
+	return &pingServiceClient{
+		ping: triple_protocol.NewClient(
+			httpClient,
+			baseURL+PingServicePingProcedure,
+			triple_protocol.WithIdempotency(triple_protocol.IdempotencyNoSideEffects),
+			triple_protocol.WithClientOptions(opts...),
+		),
+		fail: triple_protocol.NewClient(
+			httpClient,
+			baseURL+PingServiceFailProcedure,
+			opts...,
+		),
+		sum: triple_protocol.NewClient(
+			httpClient,
+			baseURL+PingServiceSumProcedure,
+			opts...,
+		),
+		countUp: triple_protocol.NewClient(
+			httpClient,
+			baseURL+PingServiceCountUpProcedure,
+			opts...,
+		),
+		cumSum: triple_protocol.NewClient(
+			httpClient,
+			baseURL+PingServiceCumSumProcedure,
+			opts...,
+		),
+	}
+}
+
+// pingServiceClient implements PingServiceClient.
+type pingServiceClient struct {
+	ping    *triple_protocol.Client
+	fail    *triple_protocol.Client
+	sum     *triple_protocol.Client
+	countUp *triple_protocol.Client
+	cumSum  *triple_protocol.Client
+}
+
+// Ping calls connect.ping.v1.PingService.Ping.
+// Ping(context.Context, *pingv1.PingRequest,*pingv1.PingResponse) (error)
+func (c *pingServiceClient) Ping(ctx context.Context, req *triple_protocol.Request, res *triple_protocol.Response) error {
+	return c.ping.CallUnary(ctx, req, res)
+}
+
+// Fail calls connect.ping.v1.PingService.Fail.
+func (c *pingServiceClient) Fail(ctx context.Context, req *triple_protocol.Request, res *triple_protocol.Response) error {
+	return c.fail.CallUnary(ctx, req, res)
+}
+
+// Sum calls connect.ping.v1.PingService.Sum.
+func (c *pingServiceClient) Sum(ctx context.Context) (*triple_protocol.ClientStreamForClient,error) {
+	return c.sum.CallClientStream(ctx)
+}
+
+// CountUp calls connect.ping.v1.PingService.CountUp.
+func (c *pingServiceClient) CountUp(ctx context.Context, req *triple_protocol.Request) (*triple_protocol.ServerStreamForClient, error) {
+	return c.countUp.CallServerStream(ctx, req)
+}
+
+// CumSum calls connect.ping.v1.PingService.CumSum.
+func (c *pingServiceClient) CumSum(ctx context.Context) (*triple_protocol.BidiStreamForClient,error) {
+	return c.cumSum.CallBidiStream(ctx)
+}
+
+// PingServiceHandler is an implementation of the connect.ping.v1.PingService service.
+type PingServiceHandler interface {
+	// Ping sends a ping to the server to determine if it's reachable.
+	Ping(context.Context, *triple_protocol.Request) (*triple_protocol.Response, error)
+	// Fail always fails.
+	Fail(context.Context, *triple_protocol.Request) (*triple_protocol.Response, error)
+	// Sum calculates the sum of the numbers sent on the stream.
+	Sum(context.Context, *triple_protocol.ClientStream) (*triple_protocol.Response, error)
+	// CountUp returns a stream of the numbers up to the given request.
+	CountUp(context.Context, *triple_protocol.Request, *triple_protocol.ServerStream) error
+	// CumSum determines the cumulative sum of all the numbers sent on the stream.
+	CumSum(context.Context, *triple_protocol.BidiStream) error
+}
+
+// NewPingServiceHandler builds an HTTP handler from the service implementation. It returns the path
+// on which to mount the handler and the handler itself.
+//
+// By default, handlers support the Connect, gRPC, and gRPC-Web protocols with the binary Protobuf
+// and JSON codecs. They also support gzip compression.
+func NewPingServiceHandler(svc PingServiceHandler, opts ...triple_protocol.HandlerOption) (string, http.Handler) {
+	pingServicePingHandler := triple_protocol.NewUnaryHandler(
+		PingServicePingProcedure, func() interface{} {
+			return &pingv1.PingRequest{}
+		},
+		svc.Ping,
+		triple_protocol.WithIdempotency(triple_protocol.IdempotencyNoSideEffects),
+		triple_protocol.WithHandlerOptions(opts...),
+	)
+	pingServiceFailHandler := triple_protocol.NewUnaryHandler(
+		PingServiceFailProcedure,
+		func() interface{} {
+			return &pingv1.FailRequest{}
+		},
+		svc.Fail,
+		opts...,
+	)
+	pingServiceSumHandler := triple_protocol.NewClientStreamHandler(
+		PingServiceSumProcedure,
+		svc.Sum,
+		opts...,
+	)
+	pingServiceCountUpHandler := triple_protocol.NewServerStreamHandler(
+		PingServiceCountUpProcedure,func() interface{} {
+			return &pingv1.CountUpRequest{}
+		},
+		svc.CountUp,
+		opts...,
+	)
+	pingServiceCumSumHandler := triple_protocol.NewBidiStreamHandler(
+		PingServiceCumSumProcedure,
+		svc.CumSum,
+		opts...,
+	)
+	return "/connect.ping.v1.PingService/", http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		switch r.URL.Path {
+		case PingServicePingProcedure:
+			pingServicePingHandler.ServeHTTP(w, r)
+		case PingServiceFailProcedure:
+			pingServiceFailHandler.ServeHTTP(w, r)
+		case PingServiceSumProcedure:
+			pingServiceSumHandler.ServeHTTP(w, r)
+		case PingServiceCountUpProcedure:
+			pingServiceCountUpHandler.ServeHTTP(w, r)
+		case PingServiceCumSumProcedure:
+			pingServiceCumSumHandler.ServeHTTP(w, r)
+		default:
+			http.NotFound(w, r)
+		}
+	})
+}
+
+// UnimplementedPingServiceHandler returns CodeUnimplemented from all methods.
+type UnimplementedPingServiceHandler struct{}
+
+func (UnimplementedPingServiceHandler) Ping(context.Context, *triple_protocol.Request) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("connect.ping.v1.PingService.Ping is not implemented"))
+}
+
+func (UnimplementedPingServiceHandler) Fail(context.Context, *triple_protocol.Request) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("connect.ping.v1.PingService.Fail is not implemented"))
+}
+
+func (UnimplementedPingServiceHandler) Sum(context.Context, *triple_protocol.ClientStream) (*triple_protocol.Response, error) {
+	return nil, triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("connect.ping.v1.PingService.Sum is not implemented"))
+}
+
+func (UnimplementedPingServiceHandler) CountUp(context.Context, *triple_protocol.Request, *triple_protocol.ServerStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("connect.ping.v1.PingService.CountUp is not implemented"))
+}
+
+func (UnimplementedPingServiceHandler) CumSum(context.Context, *triple_protocol.BidiStream) error {
+	return triple_protocol.NewError(triple_protocol.CodeUnimplemented, errors.New("connect.ping.v1.PingService.CumSum is not implemented"))
+}
diff --git a/protocol/triple/triple_protocol/internal/gen/proto/connectext/grpc/status/v1/status.pb.go b/protocol/triple/triple_protocol/internal/gen/proto/connectext/grpc/status/v1/status.pb.go
new file mode 100644
index 0000000..64b351d
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/gen/proto/connectext/grpc/status/v1/status.pb.go
@@ -0,0 +1,208 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        (unknown)
+// source: proto/connectext/grpc/status/v1/status.proto
+
+// This package is for internal use by Connect, and provides no backward
+// compatibility guarantees whatsoever.
+
+package statusv1
+
+import (
+	reflect "reflect"
+	sync "sync"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+
+	anypb "google.golang.org/protobuf/types/known/anypb"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+// See https://cloud.google.com/apis/design/errors.
+//
+// This struct must remain binary-compatible with
+// https://github.com/googleapis/googleapis/blob/master/google/rpc/status.proto.
+type Status struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Code    int32        `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"`      // a google.rpc.Code
+	Message string       `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // developer-facing, English (localize in details or client-side)
+	Details []*anypb.Any `protobuf:"bytes,3,rep,name=details,proto3" json:"details,omitempty"`
+}
+
+func (x *Status) Reset() {
+	*x = Status{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_proto_connectext_grpc_status_v1_status_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Status) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Status) ProtoMessage() {}
+
+func (x *Status) ProtoReflect() protoreflect.Message {
+	mi := &file_proto_connectext_grpc_status_v1_status_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Status.ProtoReflect.Descriptor instead.
+func (*Status) Descriptor() ([]byte, []int) {
+	return file_proto_connectext_grpc_status_v1_status_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *Status) GetCode() int32 {
+	if x != nil {
+		return x.Code
+	}
+	return 0
+}
+
+func (x *Status) GetMessage() string {
+	if x != nil {
+		return x.Message
+	}
+	return ""
+}
+
+func (x *Status) GetDetails() []*anypb.Any {
+	if x != nil {
+		return x.Details
+	}
+	return nil
+}
+
+var File_proto_connectext_grpc_status_v1_status_proto protoreflect.FileDescriptor
+
+var file_proto_connectext_grpc_status_v1_status_proto_rawDesc = []byte{
+	0x0a, 0x2c, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65,
+	0x78, 0x74, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2f, 0x76,
+	0x31, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0e,
+	0x67, 0x72, 0x70, 0x63, 0x2e, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x1a, 0x19,
+	0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f,
+	0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x22, 0x66, 0x0a, 0x06, 0x53, 0x74, 0x61,
+	0x74, 0x75, 0x73, 0x12, 0x12, 0x0a, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28,
+	0x05, 0x52, 0x04, 0x63, 0x6f, 0x64, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61,
+	0x67, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67,
+	0x65, 0x12, 0x2e, 0x0a, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03,
+	0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41, 0x6e, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c,
+	0x73, 0x42, 0xe4, 0x01, 0x0a, 0x12, 0x63, 0x6f, 0x6d, 0x2e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x73,
+	0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x42, 0x0b, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73,
+	0x50, 0x72, 0x6f, 0x74, 0x6f, 0x50, 0x01, 0x5a, 0x67, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e,
+	0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2f, 0x64, 0x75, 0x62, 0x62, 0x6f,
+	0x2d, 0x67, 0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x74, 0x72, 0x69,
+	0x70, 0x6c, 0x65, 0x2f, 0x74, 0x72, 0x69, 0x70, 0x6c, 0x65, 0x5f, 0x70, 0x72, 0x6f, 0x74, 0x6f,
+	0x63, 0x6f, 0x6c, 0x2f, 0x67, 0x65, 0x6e, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f,
+	0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x78, 0x74, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x74,
+	0x61, 0x74, 0x75, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x76, 0x31,
+	0xa2, 0x02, 0x03, 0x47, 0x53, 0x58, 0xaa, 0x02, 0x0e, 0x47, 0x72, 0x70, 0x63, 0x2e, 0x53, 0x74,
+	0x61, 0x74, 0x75, 0x73, 0x2e, 0x56, 0x31, 0xca, 0x02, 0x0e, 0x47, 0x72, 0x70, 0x63, 0x5c, 0x53,
+	0x74, 0x61, 0x74, 0x75, 0x73, 0x5c, 0x56, 0x31, 0xe2, 0x02, 0x1a, 0x47, 0x72, 0x70, 0x63, 0x5c,
+	0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x5c, 0x56, 0x31, 0x5c, 0x47, 0x50, 0x42, 0x4d, 0x65, 0x74,
+	0x61, 0x64, 0x61, 0x74, 0x61, 0xea, 0x02, 0x10, 0x47, 0x72, 0x70, 0x63, 0x3a, 0x3a, 0x53, 0x74,
+	0x61, 0x74, 0x75, 0x73, 0x3a, 0x3a, 0x56, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_proto_connectext_grpc_status_v1_status_proto_rawDescOnce sync.Once
+	file_proto_connectext_grpc_status_v1_status_proto_rawDescData = file_proto_connectext_grpc_status_v1_status_proto_rawDesc
+)
+
+func file_proto_connectext_grpc_status_v1_status_proto_rawDescGZIP() []byte {
+	file_proto_connectext_grpc_status_v1_status_proto_rawDescOnce.Do(func() {
+		file_proto_connectext_grpc_status_v1_status_proto_rawDescData = protoimpl.X.CompressGZIP(file_proto_connectext_grpc_status_v1_status_proto_rawDescData)
+	})
+	return file_proto_connectext_grpc_status_v1_status_proto_rawDescData
+}
+
+var file_proto_connectext_grpc_status_v1_status_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_proto_connectext_grpc_status_v1_status_proto_goTypes = []interface{}{
+	(*Status)(nil),    // 0: grpc.status.v1.Status
+	(*anypb.Any)(nil), // 1: google.protobuf.Any
+}
+var file_proto_connectext_grpc_status_v1_status_proto_depIdxs = []int32{
+	1, // 0: grpc.status.v1.Status.details:type_name -> google.protobuf.Any
+	1, // [1:1] is the sub-list for method output_type
+	1, // [1:1] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_proto_connectext_grpc_status_v1_status_proto_init() }
+func file_proto_connectext_grpc_status_v1_status_proto_init() {
+	if File_proto_connectext_grpc_status_v1_status_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_proto_connectext_grpc_status_v1_status_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Status); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_proto_connectext_grpc_status_v1_status_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_proto_connectext_grpc_status_v1_status_proto_goTypes,
+		DependencyIndexes: file_proto_connectext_grpc_status_v1_status_proto_depIdxs,
+		MessageInfos:      file_proto_connectext_grpc_status_v1_status_proto_msgTypes,
+	}.Build()
+	File_proto_connectext_grpc_status_v1_status_proto = out.File
+	file_proto_connectext_grpc_status_v1_status_proto_rawDesc = nil
+	file_proto_connectext_grpc_status_v1_status_proto_goTypes = nil
+	file_proto_connectext_grpc_status_v1_status_proto_depIdxs = nil
+}
diff --git a/protocol/triple/triple_protocol/internal/proto/buf.gen.yaml b/protocol/triple/triple_protocol/internal/proto/buf.gen.yaml
new file mode 100644
index 0000000..09e46ca
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/buf.gen.yaml
@@ -0,0 +1,12 @@
+version: v1
+managed:
+  enabled: true
+  go_package_prefix:
+    default: github.com/apache/dubbo-go/protocol/triple/triple_protocol/gen
+plugins:
+  - name: go
+    out: triple_protocol/gen
+    opt: paths=source_relative
+  - name: triple
+    out: triple_protocol/gen
+    opt: paths=source_relative
diff --git a/protocol/triple/triple_protocol/internal/proto/buf.yaml b/protocol/triple/triple_protocol/internal/proto/buf.yaml
new file mode 100644
index 0000000..d7e82d7
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/buf.yaml
@@ -0,0 +1,12 @@
+version: v1
+lint:
+  use:
+    - DEFAULT
+  ignore:
+    # We don't control these definitions, so we ignore lint errors.
+    - connectext/grpc/health/v1/health.proto
+    - connectext/grpc/reflection/v1alpha/reflection.proto
+    - connectext/grpc/status/v1/status.proto
+breaking:
+  use:
+    - WIRE_JSON
diff --git a/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.pb.go b/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.pb.go
new file mode 100644
index 0000000..c2e1894
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.pb.go
@@ -0,0 +1,202 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Code generated by protoc-gen-go. DO NOT EDIT.
+// versions:
+// 	protoc-gen-go v1.30.0
+// 	protoc        v3.17.3
+// source: protocol/grpc_new/triple/proto/connectext/grpc/status/v1/status.proto
+
+// This package is for internal use by Connect, and provides no backward
+// compatibility guarantees whatsoever.
+
+package statusv1
+
+import (
+	reflect "reflect"
+	sync "sync"
+)
+
+import (
+	protoreflect "google.golang.org/protobuf/reflect/protoreflect"
+
+	protoimpl "google.golang.org/protobuf/runtime/protoimpl"
+
+	anypb "google.golang.org/protobuf/types/known/anypb"
+)
+
+const (
+	// Verify that this generated code is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion)
+	// Verify that runtime/protoimpl is sufficiently up-to-date.
+	_ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20)
+)
+
+// See https://cloud.google.com/apis/design/errors.
+//
+// This struct must remain binary-compatible with
+// https://github.com/googleapis/googleapis/blob/master/google/rpc/status.proto.
+type Status struct {
+	state         protoimpl.MessageState
+	sizeCache     protoimpl.SizeCache
+	unknownFields protoimpl.UnknownFields
+
+	Code    int32        `protobuf:"varint,1,opt,name=code,proto3" json:"code,omitempty"`      // a google.rpc.Code
+	Message string       `protobuf:"bytes,2,opt,name=message,proto3" json:"message,omitempty"` // developer-facing, English (localize in details or client-side)
+	Details []*anypb.Any `protobuf:"bytes,3,rep,name=details,proto3" json:"details,omitempty"`
+}
+
+func (x *Status) Reset() {
+	*x = Status{}
+	if protoimpl.UnsafeEnabled {
+		mi := &file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_msgTypes[0]
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		ms.StoreMessageInfo(mi)
+	}
+}
+
+func (x *Status) String() string {
+	return protoimpl.X.MessageStringOf(x)
+}
+
+func (*Status) ProtoMessage() {}
+
+func (x *Status) ProtoReflect() protoreflect.Message {
+	mi := &file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_msgTypes[0]
+	if protoimpl.UnsafeEnabled && x != nil {
+		ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x))
+		if ms.LoadMessageInfo() == nil {
+			ms.StoreMessageInfo(mi)
+		}
+		return ms
+	}
+	return mi.MessageOf(x)
+}
+
+// Deprecated: Use Status.ProtoReflect.Descriptor instead.
+func (*Status) Descriptor() ([]byte, []int) {
+	return file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescGZIP(), []int{0}
+}
+
+func (x *Status) GetCode() int32 {
+	if x != nil {
+		return x.Code
+	}
+	return 0
+}
+
+func (x *Status) GetMessage() string {
+	if x != nil {
+		return x.Message
+	}
+	return ""
+}
+
+func (x *Status) GetDetails() []*anypb.Any {
+	if x != nil {
+		return x.Details
+	}
+	return nil
+}
+
+var File_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto protoreflect.FileDescriptor
+
+var file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDesc = []byte{
+	0x0a, 0x46, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x5f,
+	0x6e, 0x65, 0x77, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e, 0x65, 0x63, 0x74, 0x65, 0x78, 0x74, 0x2f, 0x67, 0x72, 0x70,
+	0x63, 0x2f, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2f, 0x76, 0x31, 0x2f, 0x73, 0x74, 0x61, 0x74,
+	0x75, 0x73, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x0e, 0x67, 0x72, 0x70, 0x63, 0x2e, 0x73,
+	0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x76, 0x31, 0x1a, 0x19, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65,
+	0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2f, 0x61, 0x6e, 0x79, 0x2e, 0x70, 0x72,
+	0x6f, 0x74, 0x6f, 0x22, 0x66, 0x0a, 0x06, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x12, 0x0a,
+	0x04, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x05, 0x52, 0x04, 0x63, 0x6f, 0x64,
+	0x65, 0x12, 0x18, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x02, 0x20, 0x01,
+	0x28, 0x09, 0x52, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x2e, 0x0a, 0x07, 0x64,
+	0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x14, 0x2e, 0x67,
+	0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x41,
+	0x6e, 0x79, 0x52, 0x07, 0x64, 0x65, 0x74, 0x61, 0x69, 0x6c, 0x73, 0x42, 0x61, 0x5a, 0x5f, 0x64,
+	0x75, 0x62, 0x62, 0x6f, 0x2e, 0x61, 0x70, 0x61, 0x63, 0x68, 0x65, 0x2e, 0x6f, 0x72, 0x67, 0x2f,
+	0x64, 0x75, 0x62, 0x62, 0x6f, 0x2d, 0x67, 0x6f, 0x2f, 0x76, 0x33, 0x2f, 0x70, 0x72, 0x6f, 0x74,
+	0x6f, 0x63, 0x6f, 0x6c, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x5f, 0x6e, 0x65, 0x77, 0x2f, 0x63, 0x6f,
+	0x6e, 0x6e, 0x65, 0x63, 0x74, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x63, 0x6f, 0x6e, 0x6e,
+	0x65, 0x63, 0x74, 0x65, 0x78, 0x74, 0x2f, 0x67, 0x72, 0x70, 0x63, 0x2f, 0x73, 0x74, 0x61, 0x74,
+	0x75, 0x73, 0x2f, 0x76, 0x31, 0x3b, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x76, 0x31, 0x62, 0x06,
+	0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
+}
+
+var (
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescOnce sync.Once
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescData = file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDesc
+)
+
+func file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescGZIP() []byte {
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescOnce.Do(func() {
+		file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescData = protoimpl.X.CompressGZIP(file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescData)
+	})
+	return file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDescData
+}
+
+var file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_msgTypes = make([]protoimpl.MessageInfo, 1)
+var file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_goTypes = []interface{}{
+	(*Status)(nil),    // 0: grpc.status.v1.Status
+	(*anypb.Any)(nil), // 1: google.protobuf.Any
+}
+var file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_depIdxs = []int32{
+	1, // 0: grpc.status.v1.Status.details:type_name -> google.protobuf.Any
+	1, // [1:1] is the sub-list for method output_type
+	1, // [1:1] is the sub-list for method input_type
+	1, // [1:1] is the sub-list for extension type_name
+	1, // [1:1] is the sub-list for extension extendee
+	0, // [0:1] is the sub-list for field type_name
+}
+
+func init() { file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_init() }
+func file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_init() {
+	if File_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto != nil {
+		return
+	}
+	if !protoimpl.UnsafeEnabled {
+		file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_msgTypes[0].Exporter = func(v interface{}, i int) interface{} {
+			switch v := v.(*Status); i {
+			case 0:
+				return &v.state
+			case 1:
+				return &v.sizeCache
+			case 2:
+				return &v.unknownFields
+			default:
+				return nil
+			}
+		}
+	}
+	type x struct{}
+	out := protoimpl.TypeBuilder{
+		File: protoimpl.DescBuilder{
+			GoPackagePath: reflect.TypeOf(x{}).PkgPath(),
+			RawDescriptor: file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDesc,
+			NumEnums:      0,
+			NumMessages:   1,
+			NumExtensions: 0,
+			NumServices:   0,
+		},
+		GoTypes:           file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_goTypes,
+		DependencyIndexes: file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_depIdxs,
+		MessageInfos:      file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_msgTypes,
+	}.Build()
+	File_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto = out.File
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_rawDesc = nil
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_goTypes = nil
+	file_protocol_grpc_new_connect_proto_connectext_grpc_status_v1_status_proto_depIdxs = nil
+}
diff --git a/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.proto b/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.proto
new file mode 100644
index 0000000..4642224
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1/status.proto
@@ -0,0 +1,32 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+syntax = "proto3";
+
+// This package is for internal use by Connect, and provides no backward
+// compatibility guarantees whatsoever.
+package grpc.status.v1;
+option go_package = "dubbo.apache.org/dubbo-go/v3/protocol/grpc_new/triple/proto/connectext/grpc/status/v1;statusv1";
+
+import "google/protobuf/any.proto";
+
+// See https://cloud.google.com/apis/design/errors.
+//
+// This struct must remain binary-compatible with
+// https://github.com/googleapis/googleapis/blob/master/google/rpc/status.proto.
+message Status {
+  int32 code = 1; // a google.rpc.Code
+  string message = 2; // developer-facing, English (localize in details or client-side)
+  repeated google.protobuf.Any details = 3;
+}
diff --git a/protocol/triple/triple_protocol/internal/proto/triple/collide/v1/collide.proto b/protocol/triple/triple_protocol/internal/proto/triple/collide/v1/collide.proto
new file mode 100644
index 0000000..dc8d711
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/triple/collide/v1/collide.proto
@@ -0,0 +1,27 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+syntax = "proto3";
+
+option go_package = "./;v1";
+
+package connect.collide.v1;
+
+message ImportRequest {}
+
+message ImportResponse {}
+
+service CollideService {
+  rpc Import(ImportRequest) returns (ImportResponse) {}
+}
diff --git a/protocol/triple/triple_protocol/internal/proto/triple/import/v1/import.proto b/protocol/triple/triple_protocol/internal/proto/triple/import/v1/import.proto
new file mode 100644
index 0000000..3241616
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/triple/import/v1/import.proto
@@ -0,0 +1,19 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+syntax = "proto3";
+
+package connect.import.v1;
+
+service ImportService {}
diff --git a/protocol/triple/triple_protocol/internal/proto/triple/ping/v1/ping.proto b/protocol/triple/triple_protocol/internal/proto/triple/ping/v1/ping.proto
new file mode 100644
index 0000000..fd8669d
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/proto/triple/ping/v1/ping.proto
@@ -0,0 +1,76 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// The canonical location for this file is
+// https://github.com/bufbuild/connect-go/blob/main/internal/proto/connect/ping/v1/ping.proto.
+syntax = "proto3";
+
+// The connect.ping.v1 package contains an echo service designed to test the
+// connect-go implementation.
+package connect.ping.v1;
+
+message PingRequest {
+  int64 number = 1;
+  string text = 2;
+}
+
+message PingResponse {
+  int64 number = 1;
+  string text = 2;
+}
+
+message FailRequest {
+  int32 code = 1;
+}
+
+message FailResponse {}
+
+message SumRequest {
+  int64 number = 1;
+}
+
+message SumResponse {
+  int64 sum = 1;
+}
+
+message CountUpRequest {
+  int64 number = 1;
+}
+
+message CountUpResponse {
+  int64 number = 1;
+}
+
+message CumSumRequest {
+  int64 number = 1;
+}
+
+message CumSumResponse {
+  int64 sum = 1;
+}
+
+service PingService {
+  // Ping sends a ping to the server to determine if it's reachable.
+  rpc Ping(PingRequest) returns (PingResponse) {
+    option idempotency_level = NO_SIDE_EFFECTS;
+  }
+  // Fail always fails.
+  rpc Fail(FailRequest) returns (FailResponse) {}
+  // Sum calculates the sum of the numbers sent on the stream.
+  rpc Sum(stream SumRequest) returns (SumResponse) {}
+  // CountUp returns a stream of the numbers up to the given request.
+  rpc CountUp(CountUpRequest) returns (stream CountUpResponse) {}
+  // CumSum determines the cumulative sum of all the numbers sent on the stream.
+  rpc CumSum(stream CumSumRequest) returns (stream CumSumResponse) {}
+}
diff --git a/protocol/triple/triple_protocol/internal/testdata/server.crt b/protocol/triple/triple_protocol/internal/testdata/server.crt
new file mode 100644
index 0000000..7917664
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/testdata/server.crt
@@ -0,0 +1,16 @@
+-----BEGIN CERTIFICATE-----
+MIICmjCCAfygAwIBAgIUGM2+eTbJp3g6o3DPtDQG3tVfL+EwCgYIKoZIzj0EAwIw
+XzELMAkGA1UEBhMCVVMxEzARBgNVBAgMCkNhbGlmb3JuaWExFjAUBgNVBAcMDVNh
+biBGcmFuY2lzY28xDjAMBgNVBAoMBXJlUlBDMRMwEQYDVQQDDApnaXRodWIuY29t
+MB4XDTIxMDczMDIwMTQzM1oXDTMxMDcyODIwMTQzM1owXzELMAkGA1UEBhMCVVMx
+EzARBgNVBAgMCkNhbGlmb3JuaWExFjAUBgNVBAcMDVNhbiBGcmFuY2lzY28xDjAM
+BgNVBAoMBXJlUlBDMRMwEQYDVQQDDApnaXRodWIuY29tMIGbMBAGByqGSM49AgEG
+BSuBBAAjA4GGAAQA4WPD74+AyZAOxAgWo58oC1JUnFy9Ln3A66rWmDPPprCJhIJ9
+i5SyXG1NxwMEIGzyFT3Bp4wWru0ogfpTxPClQ/4Aulrqisiyu4C9Ds1DRJg53E8D
+n/CKsQwUYo7MbZIrn63+77kNlJlKloUfBygZ9vQiLjhNA52A95aWRp5yNna/GvCj
+UzBRMB0GA1UdDgQWBBRm+gq9izCELNh05BdEH79AWoR9ezAfBgNVHSMEGDAWgBRm
++gq9izCELNh05BdEH79AWoR9ezAPBgNVHRMBAf8EBTADAQH/MAoGCCqGSM49BAMC
+A4GLADCBhwJCAT6Yj94euijggFrKJMcHNV7OZzFkugqiBzOI4OsjA6NfU0UExxBq
+VDuUUk2Ek3c4GWYuPvDbdx49Q+zge9Cgl3jYAkF+QrzQWIJHC2L5f5wk8488DBzb
+vs0nDV9r+drHM1KDd674y/p2sjY04PQgbNgair+BxjWxCc2QQWGw0SaWDLj/Ag==
+-----END CERTIFICATE-----
diff --git a/protocol/triple/triple_protocol/internal/testdata/server.key b/protocol/triple/triple_protocol/internal/testdata/server.key
new file mode 100644
index 0000000..0011941
--- /dev/null
+++ b/protocol/triple/triple_protocol/internal/testdata/server.key
@@ -0,0 +1,10 @@
+-----BEGIN EC PARAMETERS-----
+BgUrgQQAIw==
+-----END EC PARAMETERS-----
+-----BEGIN EC PRIVATE KEY-----
+MIHcAgEBBEIAN9tRNa4oaevnNxwZEkDTIfbcPlfQP49Q4lapXa1TUXu4Olfu6QyW
+ll3OxNIwDg54nsKSuKoaVBOKQelOJjTzyLqgBwYFK4EEACOhgYkDgYYABADhY8Pv
+j4DJkA7ECBajnygLUlScXL0ufcDrqtaYM8+msImEgn2LlLJcbU3HAwQgbPIVPcGn
+jBau7SiB+lPE8KVD/gC6WuqKyLK7gL0OzUNEmDncTwOf8IqxDBRijsxtkiufrf7v
+uQ2UmUqWhR8HKBn29CIuOE0DnYD3lpZGnnI2dr8a8A==
+-----END EC PRIVATE KEY-----
diff --git a/protocol/triple/triple_protocol/maxbytes.go b/protocol/triple/triple_protocol/maxbytes.go
new file mode 100644
index 0000000..1328482
--- /dev/null
+++ b/protocol/triple/triple_protocol/maxbytes.go
@@ -0,0 +1,32 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+//go:build go1.19
+
+package triple_protocol
+
+import (
+	"errors"
+	"fmt"
+	"net/http"
+)
+
+func asMaxBytesError(err error, tmpl string, args ...interface{}) *Error {
+	var maxBytesErr *http.MaxBytesError
+	if ok := errors.As(err, &maxBytesErr); !ok {
+		return nil
+	}
+	prefix := fmt.Sprintf(tmpl, args...)
+	return errorf(CodeResourceExhausted, "%s: exceeded %d byte http.MaxBytesReader limit", prefix, maxBytesErr.Limit)
+}
diff --git a/protocol/triple/triple_protocol/option.go b/protocol/triple/triple_protocol/option.go
new file mode 100644
index 0000000..341ddc7
--- /dev/null
+++ b/protocol/triple/triple_protocol/option.go
@@ -0,0 +1,552 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"compress/gzip"
+	"context"
+	"io"
+	"net/http"
+)
+
+// A ClientOption configures a [Client].
+//
+// In addition to any options grouped in the documentation below, remember that
+// any [Option] is also a valid ClientOption.
+type ClientOption interface {
+	applyToClient(*clientConfig)
+}
+
+// WithAcceptCompression makes a compression algorithm available to a client.
+// Clients ask servers to compress responses using any of the registered
+// algorithms. The first registered algorithm is treated as the least
+// preferred, and the last registered algorithm is the most preferred.
+//
+// It's safe to use this option liberally: servers will ignore any
+// compression algorithms they don't support. To compress requests, pair this
+// option with [WithSendCompression]. To remove support for a
+// previously-registered compression algorithm, use WithAcceptCompression with
+// nil decompressor and compressor constructors.
+//
+// Clients accept gzipped responses by default, using a compressor backed by the
+// standard library's [gzip] package with the default compression level. Use
+// [WithSendGzip] to compress requests with gzip.
+//
+// Calling WithAcceptCompression with an empty name is a no-op.
+func WithAcceptCompression(
+	name string,
+	newDecompressor func() Decompressor,
+	newCompressor func() Compressor,
+) ClientOption {
+	return &compressionOption{
+		Name:            name,
+		CompressionPool: newCompressionPool(newDecompressor, newCompressor),
+	}
+}
+
+// WithClientOptions composes multiple ClientOptions into one.
+func WithClientOptions(options ...ClientOption) ClientOption {
+	return &clientOptionsOption{options}
+}
+
+// WithTriple configures clients to use the Triple protocol.
+func WithTriple() ClientOption {
+	return &tripleOption{}
+}
+
+// WithProtoJSON configures a client to send JSON-encoded data instead of
+// binary Protobuf. It uses the standard Protobuf JSON mapping as implemented
+// by [google.golang.org/protobuf/encoding/protojson]: fields are named using
+// lowerCamelCase, zero values are omitted, missing required fields are errors,
+// enums are emitted as strings, etc.
+func WithProtoJSON() ClientOption {
+	return WithCodec(&protoJSONCodec{codecNameJSON})
+}
+
+// WithSendCompression configures the client to use the specified algorithm to
+// compress request messages. If the algorithm has not been registered using
+// [WithAcceptCompression], the client will return errors at runtime.
+//
+// Because some servers don't support compression, clients default to sending
+// uncompressed requests.
+func WithSendCompression(name string) ClientOption {
+	return &sendCompressionOption{Name: name}
+}
+
+// WithSendGzip configures the client to gzip requests. Since clients have
+// access to a gzip compressor by default, WithSendGzip doesn't require
+// [WithSendCompression].
+//
+// Some servers don't support gzip, so clients default to sending uncompressed
+// requests.
+func WithSendGzip() ClientOption {
+	return WithSendCompression(compressionGzip)
+}
+
+// A HandlerOption configures a [Handler].
+//
+// In addition to any options grouped in the documentation below, remember that
+// any [Option] is also a HandlerOption.
+type HandlerOption interface {
+	applyToHandler(*handlerConfig)
+}
+
+// WithCompression configures handlers to support a compression algorithm.
+// Clients may send messages compressed with that algorithm and/or request
+// compressed responses. The [Compressor] and [Decompressor] produced by the
+// supplied constructors must use the same algorithm. Internally, Connect pools
+// compressors and decompressors.
+//
+// By default, handlers support gzip using the standard library's
+// [compress/gzip] package at the default compression level. To remove support for
+// a previously-registered compression algorithm, use WithCompression with nil
+// decompressor and compressor constructors.
+//
+// Calling WithCompression with an empty name is a no-op.
+func WithCompression(
+	name string,
+	newDecompressor func() Decompressor,
+	newCompressor func() Compressor,
+) HandlerOption {
+	return &compressionOption{
+		Name:            name,
+		CompressionPool: newCompressionPool(newDecompressor, newCompressor),
+	}
+}
+
+// WithHandlerOptions composes multiple HandlerOptions into one.
+func WithHandlerOptions(options ...HandlerOption) HandlerOption {
+	return &handlerOptionsOption{options}
+}
+
+// WithRecover adds an interceptor that recovers from panics. The supplied
+// function receives the context, [Spec], request headers, and the recovered
+// value (which may be nil). It must return an error to send back to the
+// client. It may also log the panic, emit metrics, or execute other
+// error-handling logic. Handler functions must be safe to call concurrently.
+//
+// To preserve compatibility with [net/http]'s semantics, this interceptor
+// doesn't handle panics with [http.ErrAbortHandler].
+//
+// By default, handlers don't recover from panics. Because the standard
+// library's [http.Server] recovers from panics by default, this option isn't
+// usually necessary to prevent crashes. Instead, it helps servers collect
+// RPC-specific data during panics and send a more detailed error to
+// clients.
+func WithRecover(handle func(context.Context, Spec, http.Header, interface{}) error) HandlerOption {
+	return WithInterceptors(&recoverHandlerInterceptor{handle: handle})
+}
+
+// WithRequireConnectProtocolHeader configures the Handler to require requests
+// using the Connect RPC protocol to include the Connect-Protocol-Version
+// header. This ensures that HTTP proxies and net/http middleware can easily
+// identify valid Connect requests, even if they use a common Content-Type like
+// application/json. However, it makes ad-hoc requests with tools like cURL
+// more laborious.
+//
+// This option has no effect if the client uses the gRPC or gRPC-Web protocols.
+func WithRequireConnectProtocolHeader() HandlerOption {
+	return &requireConnectProtocolHeaderOption{}
+}
+
+// Option implements both [ClientOption] and [HandlerOption], so it can be
+// applied both client-side and server-side.
+type Option interface {
+	ClientOption
+	HandlerOption
+}
+
+// WithCodec registers a serialization method with a client or handler.
+// Handlers may have multiple codecs registered, and use whichever the client
+// chooses. Clients may only have a single codec.
+//
+// By default, handlers and clients support binary Protocol Buffer data using
+// [google.golang.org/protobuf/proto]. Handlers also support JSON by default,
+// using the standard Protobuf JSON mapping. Users with more specialized needs
+// may override the default codecs by registering a new codec under the "proto"
+// or "json" names. When supplying a custom "proto" codec, keep in mind that
+// some unexported, protocol-specific messages are serialized using Protobuf -
+// take care to fall back to the standard Protobuf implementation if
+// necessary.
+//
+// Registering a codec with an empty name is a no-op.
+func WithCodec(codec Codec) Option {
+	return &codecOption{Codec: codec}
+}
+
+// WithCompressMinBytes sets a minimum size threshold for compression:
+// regardless of compressor configuration, messages smaller than the configured
+// minimum are sent uncompressed.
+//
+// The default minimum is zero. Setting a minimum compression threshold may
+// improve overall performance, because the CPU cost of compressing very small
+// messages usually isn't worth the small reduction in network I/O.
+func WithCompressMinBytes(min int) Option {
+	return &compressMinBytesOption{Min: min}
+}
+
+// WithReadMaxBytes limits the performance impact of pathologically large
+// messages sent by the other party. For handlers, WithReadMaxBytes limits the size
+// of a message that the client can send. For clients, WithReadMaxBytes limits the
+// size of a message that the server can respond with. Limits apply to each Protobuf
+// message, not to the stream as a whole.
+//
+// Setting WithReadMaxBytes to zero allows any message size. Both clients and
+// handlers default to allowing any request size.
+//
+// Handlers may also use [http.MaxBytesHandler] to limit the total size of the
+// HTTP request stream (rather than the per-message size). Connect handles
+// [http.MaxBytesError] specially, so clients still receive errors with the
+// appropriate error code and informative messages.
+func WithReadMaxBytes(max int) Option {
+	return &readMaxBytesOption{Max: max}
+}
+
+// WithSendMaxBytes prevents sending messages too large for the client/handler
+// to handle without significant performance overhead. For handlers, WithSendMaxBytes
+// limits the size of a message that the handler can respond with. For clients,
+// WithSendMaxBytes limits the size of a message that the client can send. Limits
+// apply to each message, not to the stream as a whole.
+//
+// Setting WithSendMaxBytes to zero allows any message size. Both clients and
+// handlers default to allowing any message size.
+func WithSendMaxBytes(max int) Option {
+	return &sendMaxBytesOption{Max: max}
+}
+
+// WithIdempotency declares the idempotency of the procedure. This can determine
+// whether a procedure call can safely be retried, and may affect which request
+// modalities are allowed for a given procedure call.
+//
+// In most cases, you should not need to manually set this. It is normally set
+// by the code generator for your schema. For protobuf schemas, it can be set like this:
+//
+//	rpc Ping(PingRequest) returns (PingResponse) {
+//	  option idempotency_level = NO_SIDE_EFFECTS;
+//	}
+func WithIdempotency(idempotencyLevel IdempotencyLevel) Option {
+	return &idempotencyOption{idempotencyLevel: idempotencyLevel}
+}
+
+// WithHTTPGet allows Connect-protocol clients to use HTTP GET requests for
+// side-effect free unary RPC calls. Typically, the service schema indicates
+// which procedures are idempotent (see [WithIdempotency] for an example
+// protobuf schema). The gRPC and gRPC-Web protocols are POST-only, so this
+// option has no effect when combined with [WithGRPC] or [WithGRPCWeb].
+//
+// Using HTTP GET requests makes it easier to take advantage of CDNs, caching
+// reverse proxies, and browsers' built-in caching. Note, however, that servers
+// don't automatically set any cache headers; you can set cache headers using
+// interceptors or by adding headers in individual procedure implementations.
+//
+// By default, all requests are made as HTTP POSTs.
+func WithHTTPGet() ClientOption {
+	return &enableGet{}
+}
+
+// WithInterceptors configures a client or handler's interceptor stack. Repeated
+// WithInterceptors options are applied in order, so
+//
+//	WithInterceptors(A) + WithInterceptors(B, C) == WithInterceptors(A, B, C)
+//
+// Unary interceptors compose like an onion. The first interceptor provided is
+// the outermost layer of the onion: it acts first on the context and request,
+// and last on the response and error.
+//
+// Stream interceptors also behave like an onion: the first interceptor
+// provided is the outermost wrapper for the [StreamingClientConn] or
+// [StreamingHandlerConn]. It's the first to see sent messages and the last to
+// see received messages.
+//
+// Applied to client and handler, WithInterceptors(A, B, ..., Y, Z) produces:
+//
+//	 client.Send()       client.Receive()
+//	       |                   ^
+//	       v                   |
+//	    A ---                 --- A
+//	    B ---                 --- B
+//	    : ...                 ... :
+//	    Y ---                 --- Y
+//	    Z ---                 --- Z
+//	       |                   ^
+//	       v                   |
+//	  = = = = = = = = = = = = = = = =
+//	               network
+//	  = = = = = = = = = = = = = = = =
+//	       |                   ^
+//	       v                   |
+//	    A ---                 --- A
+//	    B ---                 --- B
+//	    : ...                 ... :
+//	    Y ---                 --- Y
+//	    Z ---                 --- Z
+//	       |                   ^
+//	       v                   |
+//	handler.Receive()   handler.Send()
+//	       |                   ^
+//	       |                   |
+//	       '-> handler logic >-'
+//
+// Note that in clients, Send handles the request message(s) and Receive
+// handles the response message(s). For handlers, it's the reverse. Depending
+// on your interceptor's logic, you may need to wrap one method in clients and
+// the other in handlers.
+func WithInterceptors(interceptors ...Interceptor) Option {
+	return &interceptorsOption{interceptors}
+}
+
+// WithOptions composes multiple Options into one.
+func WithOptions(options ...Option) Option {
+	return &optionsOption{options}
+}
+
+type clientOptionsOption struct {
+	options []ClientOption
+}
+
+func (o *clientOptionsOption) applyToClient(config *clientConfig) {
+	for _, option := range o.options {
+		option.applyToClient(config)
+	}
+}
+
+type codecOption struct {
+	Codec Codec
+}
+
+func (o *codecOption) applyToClient(config *clientConfig) {
+	if o.Codec == nil || o.Codec.Name() == "" {
+		return
+	}
+	config.Codec = o.Codec
+}
+
+func (o *codecOption) applyToHandler(config *handlerConfig) {
+	if o.Codec == nil || o.Codec.Name() == "" {
+		return
+	}
+	config.Codecs[o.Codec.Name()] = o.Codec
+}
+
+type compressionOption struct {
+	Name            string
+	CompressionPool *compressionPool
+}
+
+func (o *compressionOption) applyToClient(config *clientConfig) {
+	o.apply(&config.CompressionNames, config.CompressionPools)
+}
+
+func (o *compressionOption) applyToHandler(config *handlerConfig) {
+	o.apply(&config.CompressionNames, config.CompressionPools)
+}
+
+func (o *compressionOption) apply(configuredNames *[]string, configuredPools map[string]*compressionPool) {
+	if o.Name == "" {
+		return
+	}
+	if o.CompressionPool == nil {
+		delete(configuredPools, o.Name)
+		var names []string
+		for _, name := range *configuredNames {
+			if name == o.Name {
+				continue
+			}
+			names = append(names, name)
+		}
+		*configuredNames = names
+		return
+	}
+	configuredPools[o.Name] = o.CompressionPool
+	*configuredNames = append(*configuredNames, o.Name)
+}
+
+type compressMinBytesOption struct {
+	Min int
+}
+
+func (o *compressMinBytesOption) applyToClient(config *clientConfig) {
+	config.CompressMinBytes = o.Min
+}
+
+func (o *compressMinBytesOption) applyToHandler(config *handlerConfig) {
+	config.CompressMinBytes = o.Min
+}
+
+type readMaxBytesOption struct {
+	Max int
+}
+
+func (o *readMaxBytesOption) applyToClient(config *clientConfig) {
+	config.ReadMaxBytes = o.Max
+}
+
+func (o *readMaxBytesOption) applyToHandler(config *handlerConfig) {
+	config.ReadMaxBytes = o.Max
+}
+
+type sendMaxBytesOption struct {
+	Max int
+}
+
+func (o *sendMaxBytesOption) applyToClient(config *clientConfig) {
+	config.SendMaxBytes = o.Max
+}
+
+func (o *sendMaxBytesOption) applyToHandler(config *handlerConfig) {
+	config.SendMaxBytes = o.Max
+}
+
+type handlerOptionsOption struct {
+	options []HandlerOption
+}
+
+func (o *handlerOptionsOption) applyToHandler(config *handlerConfig) {
+	for _, option := range o.options {
+		option.applyToHandler(config)
+	}
+}
+
+type requireConnectProtocolHeaderOption struct{}
+
+func (o *requireConnectProtocolHeaderOption) applyToHandler(config *handlerConfig) {
+	config.RequireConnectProtocolHeader = true
+}
+
+type idempotencyOption struct {
+	idempotencyLevel IdempotencyLevel
+}
+
+func (o *idempotencyOption) applyToClient(config *clientConfig) {
+	config.IdempotencyLevel = o.idempotencyLevel
+}
+
+func (o *idempotencyOption) applyToHandler(config *handlerConfig) {
+	config.IdempotencyLevel = o.idempotencyLevel
+}
+
+type tripleOption struct{}
+
+func (o *tripleOption) applyToClient(config *clientConfig) {
+	config.Protocol = &protocolTriple{}
+}
+
+type enableGet struct{}
+
+func (o *enableGet) applyToClient(config *clientConfig) {
+	config.EnableGet = true
+}
+
+// withHTTPGetMaxURLSize sets the maximum allowable URL length for GET requests
+// made using the Connect protocol. It has no effect on gRPC or gRPC-Web
+// clients, since those protocols are POST-only.
+//
+// Limiting the URL size is useful as most user agents, proxies, and servers
+// have limits on the allowable length of a URL. For example, Apache and Nginx
+// limit the size of a request line to around 8 KiB, meaning that maximum
+// length of a URL is a bit smaller than this. If you run into URL size
+// limitations imposed by your network infrastructure and don't know the
+// maximum allowable size, or if you'd prefer to be cautious from the start, a
+// 4096 byte (4 KiB) limit works with most common proxies and CDNs.
+//
+// If fallback is set to true and the URL would be longer than the configured
+// maximum value, the request will be sent as an HTTP POST instead. If fallback
+// is set to false, the request will fail with [CodeResourceExhausted].
+//
+// By default, Connect-protocol clients with GET requests enabled may send a
+// URL of any size.
+func withHTTPGetMaxURLSize(bytes int, fallback bool) ClientOption {
+	return &getURLMaxBytes{Max: bytes, Fallback: fallback}
+}
+
+type getURLMaxBytes struct {
+	Max      int
+	Fallback bool
+}
+
+func (o *getURLMaxBytes) applyToClient(config *clientConfig) {
+	config.GetURLMaxBytes = o.Max
+	config.GetUseFallback = o.Fallback
+}
+
+type interceptorsOption struct {
+	Interceptors []Interceptor
+}
+
+func (o *interceptorsOption) applyToClient(config *clientConfig) {
+	config.Interceptor = o.chainWith(config.Interceptor)
+}
+
+func (o *interceptorsOption) applyToHandler(config *handlerConfig) {
+	config.Interceptor = o.chainWith(config.Interceptor)
+}
+
+func (o *interceptorsOption) chainWith(current Interceptor) Interceptor {
+	if len(o.Interceptors) == 0 {
+		return current
+	}
+	if current == nil && len(o.Interceptors) == 1 {
+		return o.Interceptors[0]
+	}
+	if current == nil && len(o.Interceptors) > 1 {
+		return newChain(o.Interceptors)
+	}
+	return newChain(append([]Interceptor{current}, o.Interceptors...))
+}
+
+type optionsOption struct {
+	options []Option
+}
+
+func (o *optionsOption) applyToClient(config *clientConfig) {
+	for _, option := range o.options {
+		option.applyToClient(config)
+	}
+}
+
+func (o *optionsOption) applyToHandler(config *handlerConfig) {
+	for _, option := range o.options {
+		option.applyToHandler(config)
+	}
+}
+
+type sendCompressionOption struct {
+	Name string
+}
+
+func (o *sendCompressionOption) applyToClient(config *clientConfig) {
+	config.RequestCompressionName = o.Name
+}
+
+func withGzip() Option {
+	return &compressionOption{
+		Name: compressionGzip,
+		CompressionPool: newCompressionPool(
+			func() Decompressor { return &gzip.Reader{} },
+			func() Compressor { return gzip.NewWriter(io.Discard) },
+		),
+	}
+}
+
+func withProtoBinaryCodec() Option {
+	return WithCodec(&protoBinaryCodec{})
+}
+
+func withProtoJSONCodecs() HandlerOption {
+	return WithHandlerOptions(
+		WithCodec(&protoJSONCodec{codecNameJSON}),
+		WithCodec(&protoJSONCodec{codecNameJSONCharsetUTF8}),
+	)
+}
diff --git a/protocol/triple/triple_protocol/protobuf_util.go b/protocol/triple/triple_protocol/protobuf_util.go
new file mode 100644
index 0000000..da24540
--- /dev/null
+++ b/protocol/triple/triple_protocol/protobuf_util.go
@@ -0,0 +1,42 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"strings"
+)
+
+// extractProtoPath returns the trailing portion of the URL's path,
+// corresponding to the Protobuf package, service, and method. It always starts
+// with a slash. Within triple, we use this as (1) Spec.Procedure and (2) the
+// path when mounting handlers on muxes.
+func extractProtoPath(path string) string {
+	segments := strings.Split(path, "/")
+	var pkg, method string
+	if len(segments) > 0 {
+		pkg = segments[0]
+	}
+	if len(segments) > 1 {
+		pkg = segments[len(segments)-2]
+		method = segments[len(segments)-1]
+	}
+	if pkg == "" {
+		return "/"
+	}
+	if method == "" {
+		return "/" + pkg
+	}
+	return "/" + pkg + "/" + method
+}
diff --git a/protocol/triple/triple_protocol/protobuf_util_test.go b/protocol/triple/triple_protocol/protobuf_util_test.go
new file mode 100644
index 0000000..3df637a
--- /dev/null
+++ b/protocol/triple/triple_protocol/protobuf_util_test.go
@@ -0,0 +1,63 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"testing"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestParseProtobufURL(t *testing.T) {
+	t.Parallel()
+	assertExtractedProtoPath(
+		t,
+		// full URL
+		"https://api.foo.com/grpc/foo.user.v1.UserService/GetUser",
+		"/foo.user.v1.UserService/GetUser",
+	)
+	assertExtractedProtoPath(
+		t,
+		// rooted path
+		"/foo.user.v1.UserService/GetUser",
+		"/foo.user.v1.UserService/GetUser",
+	)
+	assertExtractedProtoPath(
+		t,
+		// path without leading or trailing slashes
+		"foo.user.v1.UserService/GetUser",
+		"/foo.user.v1.UserService/GetUser",
+	)
+	assertExtractedProtoPath(
+		t,
+		// path with trailing slash
+		"/foo.user.v1.UserService.GetUser/",
+		"/foo.user.v1.UserService.GetUser",
+	)
+	// edge cases
+	assertExtractedProtoPath(t, "", "/")
+	assertExtractedProtoPath(t, "//", "/")
+}
+
+func assertExtractedProtoPath(tb testing.TB, inputURL, expectPath string) {
+	tb.Helper()
+	assert.Equal(
+		tb,
+		extractProtoPath(inputURL),
+		expectPath,
+	)
+}
diff --git a/protocol/triple/triple_protocol/protocol.go b/protocol/triple/triple_protocol/protocol.go
new file mode 100644
index 0000000..3cc4813
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol.go
@@ -0,0 +1,368 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"io"
+	"mime"
+	"net/http"
+	"net/url"
+	"sort"
+	"strings"
+)
+
+// The names of the Connect, gRPC, and gRPC-Web protocols (as exposed by
+// [Peer.Protocol]). Additional protocols may be added in the future.
+const (
+	ProtocolTriple  = "triple"
+	ProtocolGRPC    = "grpc"
+	ProtocolGRPCWeb = "grpcweb"
+)
+
+const (
+	headerContentType = "Content-Type"
+	headerUserAgent   = "User-Agent"
+	headerTrailer     = "Trailer"
+
+	discardLimit = 1024 * 1024 * 4 // 4MiB
+)
+
+var errNoTimeout = errors.New("no timeout")
+
+// A Protocol defines the HTTP semantics to use when sending and receiving
+// messages. It ties together codecs, compressors, and net/http to produce
+// Senders and Receivers.
+//
+// For example, triple supports the gRPC protocol using this abstraction. Among
+// many other things, the protocol implementation is responsible for
+// translating timeouts from Go contexts to HTTP and vice versa. For gRPC, it
+// converts timeouts to and from strings (for example, 10*time.Second <->
+// "10S"), and puts those strings into the "Grpc-Timeout" HTTP header. Other
+// protocols might encode durations differently, put them into a different HTTP
+// header, or ignore them entirely.
+//
+// We don't have any short-term plans to export this interface; it's just here
+// to separate the protocol-specific portions of triple from the
+// protocol-agnostic plumbing.
+type protocol interface {
+	NewHandler(*protocolHandlerParams) protocolHandler
+	NewClient(*protocolClientParams) (protocolClient, error)
+}
+
+// HandlerParams are the arguments provided to a Protocol's NewHandler
+// method, bundled into a struct to allow backward-compatible argument
+// additions. Protocol implementations should take care to use the supplied
+// Spec rather than constructing their own, since new fields may have been
+// added.
+type protocolHandlerParams struct {
+	Spec                        Spec
+	Codecs                      readOnlyCodecs
+	CompressionPools            readOnlyCompressionPools
+	CompressMinBytes            int
+	BufferPool                  *bufferPool
+	ReadMaxBytes                int
+	SendMaxBytes                int
+	RequireTripleProtocolHeader bool
+	IdempotencyLevel            IdempotencyLevel
+}
+
+// Handler is the server side of a protocol. HTTP handlers typically support
+// multiple protocols, codecs, and compressors.
+type protocolHandler interface {
+	// Methods is the set of HTTP methods the protocol can handle.
+	Methods() map[string]struct{}
+
+	// ContentTypes is the set of HTTP Content-Types that the protocol can
+	// handle.
+	ContentTypes() map[string]struct{}
+
+	// SetTimeout runs before NewStream. Implementations may inspect the HTTP
+	// request, parse any timeout set by the client, and return a modified
+	// context and cancellation function.
+	//
+	// If the client didn't send a timeout, SetTimeout should return the
+	// request's context, a nil cancellation function, and a nil error.
+	SetTimeout(*http.Request) (context.Context, context.CancelFunc, error)
+
+	// CanHandlePayload returns true if the protocol can handle an HTTP request.
+	// This is called after the request method is validated, so we only need to
+	// be concerned with the content type/payload specifically.
+	CanHandlePayload(*http.Request, string) bool
+
+	// NewConn constructs a HandlerConn for the message exchange.
+	NewConn(http.ResponseWriter, *http.Request) (handlerConnCloser, bool)
+}
+
+// ClientParams are the arguments provided to a Protocol's NewClient method,
+// bundled into a struct to allow backward-compatible argument additions.
+// Protocol implementations should take care to use the supplied Spec rather
+// than constructing their own, since new fields may have been added.
+type protocolClientParams struct {
+	CompressionName  string
+	CompressionPools readOnlyCompressionPools
+	Codec            Codec
+	CompressMinBytes int
+	HTTPClient       HTTPClient
+	URL              *url.URL
+	BufferPool       *bufferPool
+	ReadMaxBytes     int
+	SendMaxBytes     int
+	EnableGet        bool
+	GetURLMaxBytes   int
+	GetUseFallback   bool
+	// The gRPC family of protocols always needs access to a Protobuf codec to
+	// marshal and unmarshal errors.
+	Protobuf Codec
+}
+
+// Client is the client side of a protocol. HTTP clients typically use a single
+// protocol, codec, and compressor to send requests.
+type protocolClient interface {
+	// Peer describes the server for the RPC.
+	Peer() Peer
+
+	// WriteRequestHeader writes any protocol-specific request headers.
+	WriteRequestHeader(StreamType, http.Header)
+
+	// NewConn constructs a StreamingClientConn for the message exchange.
+	//
+	// Implementations should assume that the supplied HTTP headers have already
+	// been populated by WriteRequestHeader. When constructing a stream for a
+	// unary call, implementations may assume that the Sender's Send and Close
+	// methods return before the Receiver's Receive or Close methods are called.
+	NewConn(context.Context, Spec, http.Header) StreamingClientConn
+}
+
+// errorTranslatingHandlerConnCloser wraps a handlerConnCloser to ensure that
+// we always return coded errors to users and write coded errors to the
+// network.
+//
+// It's used in protocol implementations.
+type errorTranslatingHandlerConnCloser struct {
+	handlerConnCloser
+
+	toWire   func(error) error
+	fromWire func(error) error
+}
+
+func (hc *errorTranslatingHandlerConnCloser) Send(msg interface{}) error {
+	return hc.fromWire(hc.handlerConnCloser.Send(msg))
+}
+
+func (hc *errorTranslatingHandlerConnCloser) Receive(msg interface{}) error {
+	return hc.fromWire(hc.handlerConnCloser.Receive(msg))
+}
+
+func (hc *errorTranslatingHandlerConnCloser) Close(err error) error {
+	closeErr := hc.handlerConnCloser.Close(hc.toWire(err))
+	return hc.fromWire(closeErr)
+}
+
+// errorTranslatingClientConn wraps a StreamingClientConn to make sure that we always
+// return coded errors from clients.
+//
+// It's used in protocol implementations.
+type errorTranslatingClientConn struct {
+	StreamingClientConn
+
+	fromWire func(error) error
+}
+
+func (cc *errorTranslatingClientConn) Send(msg interface{}) error {
+	return cc.fromWire(cc.StreamingClientConn.Send(msg))
+}
+
+func (cc *errorTranslatingClientConn) Receive(msg interface{}) error {
+	return cc.fromWire(cc.StreamingClientConn.Receive(msg))
+}
+
+func (cc *errorTranslatingClientConn) CloseRequest() error {
+	return cc.fromWire(cc.StreamingClientConn.CloseRequest())
+}
+
+func (cc *errorTranslatingClientConn) CloseResponse() error {
+	return cc.fromWire(cc.StreamingClientConn.CloseResponse())
+}
+
+// wrapHandlerConnWithCodedErrors ensures that we (1) automatically code
+// context-related errors correctly when writing them to the network, and (2)
+// return *Errors from all exported APIs.
+func wrapHandlerConnWithCodedErrors(conn handlerConnCloser) handlerConnCloser {
+	return &errorTranslatingHandlerConnCloser{
+		handlerConnCloser: conn,
+		toWire:            wrapIfContextError,
+		fromWire:          wrapIfUncoded,
+	}
+}
+
+// wrapClientConnWithCodedErrors ensures that we always return *Errors from
+// public APIs.
+func wrapClientConnWithCodedErrors(conn StreamingClientConn) StreamingClientConn {
+	return &errorTranslatingClientConn{
+		StreamingClientConn: conn,
+		fromWire:            wrapIfUncoded,
+	}
+}
+
+func sortedAcceptPostValue(handlers []protocolHandler) string {
+	contentTypes := make(map[string]struct{})
+	for _, handler := range handlers {
+		for contentType := range handler.ContentTypes() {
+			contentTypes[contentType] = struct{}{}
+		}
+	}
+	accept := make([]string, 0, len(contentTypes))
+	for ct := range contentTypes {
+		accept = append(accept, ct)
+	}
+	sort.Strings(accept)
+	return strings.Join(accept, ", ")
+}
+
+func sortedAllowMethodValue(handlers []protocolHandler) string {
+	methods := make(map[string]struct{})
+	for _, handler := range handlers {
+		for method := range handler.Methods() {
+			methods[method] = struct{}{}
+		}
+	}
+	allow := make([]string, 0, len(methods))
+	for ct := range methods {
+		allow = append(allow, ct)
+	}
+	sort.Strings(allow)
+	return strings.Join(allow, ", ")
+}
+
+func isCommaOrSpace(c rune) bool {
+	return c == ',' || c == ' '
+}
+
+func discard(reader io.Reader) error {
+	if lr, ok := reader.(*io.LimitedReader); ok {
+		_, err := io.Copy(io.Discard, lr)
+		return err
+	}
+	// We don't want to get stuck throwing data away forever, so limit how much
+	// we're willing to do here.
+	lr := &io.LimitedReader{R: reader, N: discardLimit}
+	_, err := io.Copy(io.Discard, lr)
+	return err
+}
+
+// negotiateCompression determines and validates the request compression and
+// response compression using the available compressors and protocol-specific
+// Content-Encoding and Accept-Encoding headers.
+func negotiateCompression( //nolint:nonamedreturns
+	availableCompressors readOnlyCompressionPools,
+	sent, accept string,
+) (requestCompression, responseCompression string, clientVisibleErr *Error) {
+	requestCompression = compressionIdentity
+	if sent != "" && sent != compressionIdentity {
+		// We default to identity, so we only care if the client sends something
+		// other than the empty string or compressIdentity.
+		if availableCompressors.Contains(sent) {
+			requestCompression = sent
+		} else {
+			// To comply with
+			// https://github.com/grpc/grpc/blob/master/doc/compression.md and the
+			// Connect protocol, we should return CodeUnimplemented and specify
+			// acceptable compression(s) (in addition to setting the a
+			// protocol-specific accept-encoding header).
+			return "", "", errorf(
+				CodeUnimplemented,
+				"unknown compression %q: supported encodings are %v",
+				sent, availableCompressors.CommaSeparatedNames(),
+			)
+		}
+	}
+	// Support asymmetric compression. This logic follows
+	// https://github.com/grpc/grpc/blob/master/doc/compression.md and common
+	// sense.
+	responseCompression = requestCompression
+	// If we're not already planning to compress the response, check whether the
+	// client requested a compression algorithm we support.
+	if responseCompression == compressionIdentity && accept != "" {
+		for _, name := range strings.FieldsFunc(accept, isCommaOrSpace) {
+			if availableCompressors.Contains(name) {
+				// We found a mutually supported compression algorithm. Unlike standard
+				// HTTP, there's no preference weighting, so can bail out immediately.
+				responseCompression = name
+				break
+			}
+		}
+	}
+	return requestCompression, responseCompression, nil
+}
+
+// checkServerStreamsCanFlush ensures that bidi and server streaming handlers
+// have received an http.ResponseWriter that implements http.Flusher, since
+// they must flush data after sending each message.
+func checkServerStreamsCanFlush(spec Spec, responseWriter http.ResponseWriter) *Error {
+	requiresFlusher := (spec.StreamType & StreamTypeServer) == StreamTypeServer
+	if _, flushable := responseWriter.(http.Flusher); requiresFlusher && !flushable {
+		return NewError(CodeInternal, fmt.Errorf("%T does not implement http.Flusher", responseWriter))
+	}
+	return nil
+}
+
+func flushResponseWriter(w http.ResponseWriter) {
+	if f, ok := w.(http.Flusher); ok {
+		f.Flush()
+	}
+}
+
+func canonicalizeContentType(contentType string) string {
+	// Typically, clients send Content-Type in canonical form, without
+	// parameters. In those cases, we'd like to avoid parsing and
+	// canonicalization overhead.
+	//
+	// See https://www.rfc-editor.org/rfc/rfc2045.html#section-5.1 for a full
+	// grammar.
+	var slashes int
+	for _, r := range contentType {
+		switch {
+		case r >= 'a' && r <= 'z':
+		case r == '.' || r == '+' || r == '-':
+		case r == '/':
+			slashes++
+		default:
+			return canonicalizeContentTypeSlow(contentType)
+		}
+	}
+	if slashes == 1 {
+		return contentType
+	}
+	return canonicalizeContentTypeSlow(contentType)
+}
+
+func canonicalizeContentTypeSlow(contentType string) string {
+	base, params, err := mime.ParseMediaType(contentType)
+	if err != nil {
+		return contentType
+	}
+	// According to RFC 9110 Section 8.3.2, the charset parameter value should be treated as case-insensitive.
+	// mime.FormatMediaType canonicalizes parameter names, but not parameter values,
+	// because the case sensitivity of a parameter value depends on its semantics.
+	// Therefore, the charset parameter value should be canonicalized here.
+	// ref.) https://httpwg.org/specs/rfc9110.html#rfc.section.8.3.2
+	if charset, ok := params["charset"]; ok {
+		params["charset"] = strings.ToLower(charset)
+	}
+	return mime.FormatMediaType(base, params)
+}
diff --git a/protocol/triple/triple_protocol/protocol_grpc.go b/protocol/triple/triple_protocol/protocol_grpc.go
new file mode 100644
index 0000000..555d295
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol_grpc.go
@@ -0,0 +1,921 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bufio"
+	"context"
+	"errors"
+	"fmt"
+	"io"
+	"math"
+	"net/http"
+	"net/textproto"
+	"runtime"
+	"strconv"
+	"strings"
+	"time"
+	"unicode/utf8"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/proto/connectext/grpc/status/v1"
+)
+
+// protocol specification headers
+const (
+	grpcHeaderCompression       = "Grpc-Encoding"
+	grpcHeaderAcceptCompression = "Grpc-Accept-Encoding"
+	grpcHeaderTimeout           = "Grpc-Timeout"
+	grpcHeaderStatus            = "Grpc-Status"
+	grpcHeaderMessage           = "Grpc-Message"
+	grpcHeaderDetails           = "Grpc-Status-Details-Bin"
+
+	grpcFlagEnvelopeTrailer = 0b10000000
+
+	grpcTimeoutMaxHours = math.MaxInt64 / int64(time.Hour) // how many hours fit into a time.Duration?
+	grpcMaxTimeoutChars = 8                                // from gRPC protocol
+
+	grpcContentTypeDefault = "application/grpc"
+	grpcContentTypePrefix  = grpcContentTypeDefault + "+"
+)
+
+var (
+	grpcTimeoutUnits = []struct {
+		size time.Duration
+		char byte
+	}{
+		{time.Nanosecond, 'n'},
+		{time.Microsecond, 'u'},
+		{time.Millisecond, 'm'},
+		{time.Second, 'S'},
+		{time.Minute, 'M'},
+		{time.Hour, 'H'},
+	}
+	grpcTimeoutUnitLookup = make(map[byte]time.Duration)
+	grpcAllowedMethods    = map[string]struct{}{
+		http.MethodPost: {},
+	}
+	errTrailersWithoutGRPCStatus = fmt.Errorf("gRPC protocol error: no %s trailer", grpcHeaderStatus)
+
+	// defaultGrpcUserAgent follows
+	// https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#user-agents:
+	//
+	//	While the protocol does not require a user-agent to function it is recommended
+	//	that clients provide a structured user-agent string that provides a basic
+	//	description of the calling library, version & platform to facilitate issue diagnosis
+	//	in heterogeneous environments. The following structure is recommended to library developers:
+	//
+	//	User-Agent → "grpc-" Language ?("-" Variant) "/" Version ?( " ("  *(AdditionalProperty ";") ")" )
+	defaultGrpcUserAgent = fmt.Sprintf("grpc-go-triple/%s (%s)", Version, runtime.Version())
+)
+
+func init() {
+	for _, pair := range grpcTimeoutUnits {
+		grpcTimeoutUnitLookup[pair.char] = pair.size
+	}
+}
+
+type protocolGRPC struct{}
+
+// for server side
+
+// NewHandler implements protocol, so it must return an interface.
+func (g *protocolGRPC) NewHandler(params *protocolHandlerParams) protocolHandler {
+	bare, prefix := grpcContentTypeDefault, grpcContentTypePrefix
+	contentTypes := make(map[string]struct{})
+	for _, name := range params.Codecs.Names() {
+		contentTypes[canonicalizeContentType(prefix+name)] = struct{}{}
+	}
+	// default codec
+	if params.Codecs.Get(codecNameProto) != nil {
+		contentTypes[bare] = struct{}{}
+	}
+	return &grpcHandler{
+		protocolHandlerParams: *params,
+		accept:                contentTypes,
+	}
+}
+
+// for client side
+
+// NewClient implements protocol, so it must return an interface.
+func (g *protocolGRPC) NewClient(params *protocolClientParams) (protocolClient, error) {
+	peer := newPeerFromURL(params.URL, ProtocolGRPC)
+	return &grpcClient{
+		protocolClientParams: *params,
+		peer:                 peer,
+	}, nil
+}
+
+type grpcHandler struct {
+	protocolHandlerParams
+
+	accept map[string]struct{}
+}
+
+func (g *grpcHandler) Methods() map[string]struct{} {
+	return grpcAllowedMethods
+}
+
+func (g *grpcHandler) ContentTypes() map[string]struct{} {
+	return g.accept
+}
+
+func (*grpcHandler) SetTimeout(request *http.Request) (context.Context, context.CancelFunc, error) {
+	timeout, err := grpcParseTimeout(getHeaderCanonical(request.Header, grpcHeaderTimeout))
+	// not set or great than the threshold
+	if err != nil && !errors.Is(err, errNoTimeout) {
+		// Errors here indicate that the client sent an invalid timeout header, so
+		// the error text is safe to send back.
+		return nil, nil, NewError(CodeInvalidArgument, err)
+	} else if err != nil {
+		// err wraps errNoTimeout, nothing to do.
+		return request.Context(), nil, nil //nolint:nilerr
+	}
+	ctx, cancel := context.WithTimeout(request.Context(), timeout)
+	return ctx, cancel, nil
+}
+
+func (g *grpcHandler) CanHandlePayload(request *http.Request, contentType string) bool {
+	_, ok := g.accept[contentType]
+	return ok
+}
+
+// NewConn is the key function which is responsible for marshal and unmarshal
+func (g *grpcHandler) NewConn(
+	responseWriter http.ResponseWriter,
+	request *http.Request,
+) (handlerConnCloser, bool) {
+	// We need to parse metadata before entering the interceptor stack; we'll
+	// send the error to the client later on.
+	requestCompression, responseCompression, failed := negotiateCompression(
+		g.CompressionPools,
+		getHeaderCanonical(request.Header, grpcHeaderCompression),
+		getHeaderCanonical(request.Header, grpcHeaderAcceptCompression),
+	)
+	if failed == nil {
+		failed = checkServerStreamsCanFlush(g.Spec, responseWriter)
+	}
+
+	// keep ready for headers of response
+
+	// Write any remaining headers here:
+	// (1) any writes to the stream will implicitly send the headers, so we
+	// should get all of gRPC's required response headers ready.
+	// (2) interceptors should be able to see these headers.
+	//
+	// Since we know that these header keys are already in canonical form, we can
+	// skip the normalization in Header.Set.
+	header := responseWriter.Header()
+	header[headerContentType] = []string{getHeaderCanonical(request.Header, headerContentType)}
+	header[grpcHeaderAcceptCompression] = []string{g.CompressionPools.CommaSeparatedNames()}
+	if responseCompression != compressionIdentity {
+		header[grpcHeaderCompression] = []string{responseCompression}
+	}
+
+	// content-type -> codecName -> codec
+	codecName := grpcCodecFromContentType(getHeaderCanonical(request.Header, headerContentType))
+	codec := g.Codecs.Get(codecName) // handler.go guarantees this is not nil
+	protocolName := ProtocolGRPC
+	conn := wrapHandlerConnWithCodedErrors(&grpcHandlerConn{
+		spec: g.Spec,
+		peer: Peer{
+			Addr:     request.RemoteAddr,
+			Protocol: protocolName,
+		},
+		bufferPool: g.BufferPool,
+		protobuf:   g.Codecs.Protobuf(), // for errors
+		marshaler: grpcMarshaler{
+			envelopeWriter: envelopeWriter{
+				writer:           responseWriter,
+				compressionPool:  g.CompressionPools.Get(responseCompression),
+				codec:            codec,
+				compressMinBytes: g.CompressMinBytes,
+				bufferPool:       g.BufferPool,
+				sendMaxBytes:     g.SendMaxBytes,
+			},
+		},
+		responseWriter:  responseWriter,
+		responseHeader:  make(http.Header),
+		responseTrailer: make(http.Header),
+		request:         request,
+		unmarshaler: grpcUnmarshaler{
+			envelopeReader: envelopeReader{
+				reader:          request.Body,
+				codec:           codec,
+				compressionPool: g.CompressionPools.Get(requestCompression),
+				bufferPool:      g.BufferPool,
+				readMaxBytes:    g.ReadMaxBytes,
+			},
+		},
+	})
+	if failed != nil {
+		// Negotiation failed, so we can't establish a stream.
+		_ = conn.Close(failed)
+		return nil, false
+	}
+	return conn, true
+}
+
+type grpcClient struct {
+	protocolClientParams
+
+	peer Peer
+}
+
+func (g *grpcClient) Peer() Peer {
+	return g.peer
+}
+
+func (g *grpcClient) WriteRequestHeader(_ StreamType, header http.Header) {
+	// We know these header keys are in canonical form, so we can bypass all the
+	// checks in Header.Set.
+	if getHeaderCanonical(header, headerUserAgent) == "" {
+		header[headerUserAgent] = []string{defaultGrpcUserAgent}
+	}
+	header[headerContentType] = []string{grpcContentTypeFromCodecName(g.Codec.Name())}
+	// gRPC handles compression on a per-message basis, so we don't want to
+	// compress the whole stream. By default, http.Client will ask the server
+	// to gzip the stream if we don't set Accept-Encoding.
+	header["Accept-Encoding"] = []string{compressionIdentity}
+	if g.CompressionName != "" && g.CompressionName != compressionIdentity {
+		header[grpcHeaderCompression] = []string{g.CompressionName}
+	}
+	if acceptCompression := g.CompressionPools.CommaSeparatedNames(); acceptCompression != "" {
+		header[grpcHeaderAcceptCompression] = []string{acceptCompression}
+	}
+	// The gRPC-HTTP2 specification requires this - it flushes out proxies that
+	// don't support HTTP trailers.
+	header["Te"] = []string{"trailers"}
+}
+
+func (g *grpcClient) NewConn(
+	ctx context.Context,
+	spec Spec,
+	header http.Header,
+) StreamingClientConn {
+	if deadline, ok := ctx.Deadline(); ok {
+		if encodedDeadline, err := grpcEncodeTimeout(time.Until(deadline)); err == nil {
+			// Tests verify that the error in encodeTimeout is unreachable, so we
+			// don't need to handle the error case.
+			header[grpcHeaderTimeout] = []string{encodedDeadline}
+		}
+	}
+	duplexCall := newDuplexHTTPCall(
+		ctx,
+		g.HTTPClient,
+		g.URL,
+		spec,
+		header,
+	)
+	conn := &grpcClientConn{
+		spec:             spec,
+		peer:             g.Peer(),
+		duplexCall:       duplexCall,
+		compressionPools: g.CompressionPools,
+		bufferPool:       g.BufferPool,
+		protobuf:         g.Protobuf,
+		marshaler: grpcMarshaler{
+			envelopeWriter: envelopeWriter{
+				writer:           duplexCall,
+				compressionPool:  g.CompressionPools.Get(g.CompressionName),
+				codec:            g.Codec,
+				compressMinBytes: g.CompressMinBytes,
+				bufferPool:       g.BufferPool,
+				sendMaxBytes:     g.SendMaxBytes,
+			},
+		},
+		unmarshaler: grpcUnmarshaler{
+			envelopeReader: envelopeReader{
+				reader:       duplexCall,
+				codec:        g.Codec,
+				bufferPool:   g.BufferPool,
+				readMaxBytes: g.ReadMaxBytes,
+			},
+		},
+		responseHeader:  make(http.Header),
+		responseTrailer: make(http.Header),
+	}
+	duplexCall.SetValidateResponse(conn.validateResponse)
+	conn.readTrailers = func(_ *grpcUnmarshaler, call *duplexHTTPCall) http.Header {
+		// To access HTTP trailers, we need to read the body to EOF.
+		_ = discard(call)
+		return call.ResponseTrailer()
+	}
+	return wrapClientConnWithCodedErrors(conn)
+}
+
+// grpcClientConn works for both gRPC and gRPC-Web.
+type grpcClientConn struct {
+	spec             Spec
+	peer             Peer
+	duplexCall       *duplexHTTPCall
+	compressionPools readOnlyCompressionPools
+	bufferPool       *bufferPool
+	protobuf         Codec // for errors
+	marshaler        grpcMarshaler
+	unmarshaler      grpcUnmarshaler
+	responseHeader   http.Header
+	responseTrailer  http.Header
+	readTrailers     func(*grpcUnmarshaler, *duplexHTTPCall) http.Header
+}
+
+func (cc *grpcClientConn) Spec() Spec {
+	return cc.spec
+}
+
+func (cc *grpcClientConn) Peer() Peer {
+	return cc.peer
+}
+
+func (cc *grpcClientConn) Send(msg interface{}) error {
+	if err := cc.marshaler.Marshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (cc *grpcClientConn) RequestHeader() http.Header {
+	return cc.duplexCall.Header()
+}
+
+func (cc *grpcClientConn) CloseRequest() error {
+	return cc.duplexCall.CloseWrite()
+}
+
+func (cc *grpcClientConn) Receive(msg interface{}) error {
+	cc.duplexCall.BlockUntilResponseReady()
+	err := cc.unmarshaler.Unmarshal(msg)
+	if err == nil {
+		return nil
+	}
+	if getHeaderCanonical(cc.responseHeader, grpcHeaderStatus) != "" {
+		// We got what gRPC calls a trailers-only response, which puts the trailing
+		// metadata (including errors) into HTTP headers. validateResponse has
+		// already extracted the error.
+		return err
+	}
+	// See if the server sent an explicit error in the HTTP or gRPC-Web trailers.
+	mergeHeaders(
+		cc.responseTrailer,
+		cc.readTrailers(&cc.unmarshaler, cc.duplexCall),
+	)
+	serverErr := grpcErrorFromTrailer(cc.bufferPool, cc.protobuf, cc.responseTrailer)
+	if serverErr != nil && (errors.Is(err, io.EOF) || !errors.Is(serverErr, errTrailersWithoutGRPCStatus)) {
+		// We've either:
+		//   - Cleanly read until the end of the response body and *not* received
+		//   gRPC status trailers, which is a protocol error, or
+		//   - Received an explicit error from the server.
+		//
+		// This is expected from a protocol perspective, but receiving trailers
+		// means that we're _not_ getting a message. For users to realize that
+		// the stream has ended, Receive must return an error.
+		serverErr.meta = cc.responseHeader.Clone()
+		mergeHeaders(serverErr.meta, cc.responseTrailer)
+		cc.duplexCall.SetError(serverErr)
+		return serverErr
+	}
+	// This was probably an error converting the bytes to a message or an error
+	// reading from the network. We're going to return it to the
+	// user, but we also want to setResponseError so Send errors out.
+	cc.duplexCall.SetError(err)
+	return err
+}
+
+func (cc *grpcClientConn) ResponseHeader() http.Header {
+	cc.duplexCall.BlockUntilResponseReady()
+	return cc.responseHeader
+}
+
+func (cc *grpcClientConn) ResponseTrailer() http.Header {
+	cc.duplexCall.BlockUntilResponseReady()
+	return cc.responseTrailer
+}
+
+func (cc *grpcClientConn) CloseResponse() error {
+	return cc.duplexCall.CloseRead()
+}
+
+func (cc *grpcClientConn) validateResponse(response *http.Response) *Error {
+	if err := grpcValidateResponse(
+		response,
+		cc.responseHeader,
+		cc.responseTrailer,
+		cc.compressionPools,
+		cc.bufferPool,
+		cc.protobuf,
+	); err != nil {
+		return err
+	}
+	compression := getHeaderCanonical(response.Header, grpcHeaderCompression)
+	cc.unmarshaler.envelopeReader.compressionPool = cc.compressionPools.Get(compression)
+	return nil
+}
+
+// connection based on compression, codec
+type grpcHandlerConn struct {
+	spec            Spec
+	peer            Peer
+	web             bool
+	bufferPool      *bufferPool
+	protobuf        Codec // for errors
+	marshaler       grpcMarshaler
+	responseWriter  http.ResponseWriter
+	responseHeader  http.Header
+	responseTrailer http.Header
+	wroteToBody     bool
+	request         *http.Request
+	unmarshaler     grpcUnmarshaler
+}
+
+func (hc *grpcHandlerConn) Spec() Spec {
+	return hc.spec
+}
+
+func (hc *grpcHandlerConn) Peer() Peer {
+	return hc.peer
+}
+
+// Receive delegated receive and unmarshal processes to unmarshaler
+func (hc *grpcHandlerConn) Receive(msg interface{}) error {
+	if err := hc.unmarshaler.Unmarshal(msg); err != nil {
+		return err // already coded
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (hc *grpcHandlerConn) RequestHeader() http.Header {
+	return hc.request.Header
+}
+
+func (hc *grpcHandlerConn) Send(msg interface{}) error {
+	defer flushResponseWriter(hc.responseWriter)
+	if !hc.wroteToBody {
+		mergeHeaders(hc.responseWriter.Header(), hc.responseHeader)
+		hc.wroteToBody = true
+	}
+	if err := hc.marshaler.Marshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (hc *grpcHandlerConn) ResponseHeader() http.Header {
+	return hc.responseHeader
+}
+
+func (hc *grpcHandlerConn) ResponseTrailer() http.Header {
+	return hc.responseTrailer
+}
+
+func (hc *grpcHandlerConn) Close(err error) (retErr error) {
+	defer func() {
+		// We don't want to copy unread portions of the body to /dev/null here: if
+		// the client hasn't closed the request body, we'll block until the server
+		// timeout kicks in. This could happen because the client is malicious, but
+		// a well-intentioned client may just not expect the server to be returning
+		// an error for a streaming RPC. Better to accept that we can't always reuse
+		// TCP connections.
+		closeErr := hc.request.Body.Close()
+		if retErr == nil {
+			retErr = closeErr
+		}
+	}()
+	defer flushResponseWriter(hc.responseWriter)
+	// If we haven't written the headers yet, do so.
+	if !hc.wroteToBody {
+		mergeHeaders(hc.responseWriter.Header(), hc.responseHeader)
+	}
+	// gRPC always sends the error's code, message, details, and metadata as
+	// trailing metadata. The Connect protocol doesn't do this, so we don't want
+	// to mutate the trailers map that the user sees.
+	mergedTrailers := make(
+		http.Header,
+		len(hc.responseTrailer)+2, // always make space for status & message
+	)
+	mergeHeaders(mergedTrailers, hc.responseTrailer)
+	grpcErrorToTrailer(hc.bufferPool, mergedTrailers, hc.protobuf, err)
+	if hc.web && !hc.wroteToBody {
+		// We're using gRPC-Web and we haven't yet written to the body. Since we're
+		// not sending any response messages, the gRPC specification calls this a
+		// "trailers-only" response. Under those circumstances, the gRPC-Web spec
+		// says that implementations _may_ send trailing metadata as HTTP headers
+		// instead. Envoy is the canonical implementation of the gRPC-Web protocol,
+		// so we emulate Envoy's behavior and put the trailing metadata in the HTTP
+		// headers.
+		mergeHeaders(hc.responseWriter.Header(), mergedTrailers)
+		return nil
+	}
+	if hc.web {
+		// We're using gRPC-Web and we've already sent the headers, so we write
+		// trailing metadata to the HTTP body.
+		if err := hc.marshaler.MarshalWebTrailers(mergedTrailers); err != nil {
+			return err
+		}
+		return nil // must be a literal nil: nil *Error is a non-nil error
+	}
+	// We're using standard gRPC. Even if we haven't written to the body and
+	// we're sending a "trailers-only" response, we must send trailing metadata
+	// as HTTP trailers. (If we had frame-level control of the HTTP/2 layer, we
+	// could send trailers-only responses as a single HEADER frame and no DATA
+	// frames, but net/http doesn't expose APIs that low-level.)
+	if !hc.wroteToBody {
+		// This block works around a bug in x/net/http2. Until Go 1.20, trailers
+		// written using http.TrailerPrefix were only sent if either (1) there's
+		// data in the body, or (2) the innermost http.ResponseWriter is flushed.
+		// To ensure that we always send a valid gRPC response, even if the user
+		// has wrapped the response writer in net/http middleware that doesn't
+		// implement http.Flusher, we must pre-declare our HTTP trailers. We can
+		// remove this when Go 1.21 ships and we drop support for Go 1.19.
+		for key := range mergedTrailers {
+			addHeaderCanonical(hc.responseWriter.Header(), headerTrailer, key)
+		}
+		hc.responseWriter.WriteHeader(http.StatusOK)
+		for key, values := range mergedTrailers {
+			for _, value := range values {
+				// These are potentially user-supplied, so we can't assume they're in
+				// canonical form. Don't use addHeaderCanonical.
+				hc.responseWriter.Header().Add(key, value)
+			}
+		}
+		return nil
+	}
+	// In net/http's ResponseWriter API, we send HTTP trailers by writing to the
+	// headers map with a special prefix. This prefixing is an implementation
+	// detail, so we should hide it and _not_ mutate the user-visible headers.
+	//
+	// Note that this is _very_ finicky and difficult to test with net/http,
+	// since correctness depends on low-level framing details. Breaking this
+	// logic breaks Envoy's gRPC-Web translation.
+	for key, values := range mergedTrailers {
+		for _, value := range values {
+			// These are potentially user-supplied, so we can't assume they're in
+			// canonical form. Don't use addHeaderCanonical.
+			hc.responseWriter.Header().Add(http.TrailerPrefix+key, value)
+		}
+	}
+	return nil
+}
+
+type grpcMarshaler struct {
+	envelopeWriter
+}
+
+func (m *grpcMarshaler) MarshalWebTrailers(trailer http.Header) *Error {
+	raw := m.envelopeWriter.bufferPool.Get()
+	defer m.envelopeWriter.bufferPool.Put(raw)
+	for key, values := range trailer {
+		// Per the Go specification, keys inserted during iteration may be produced
+		// later in the iteration or may be skipped. For safety, avoid mutating the
+		// map if the key is already lower-cased.
+		lower := strings.ToLower(key)
+		if key == lower {
+			continue
+		}
+		delete(trailer, key)
+		trailer[lower] = values
+	}
+	if err := trailer.Write(raw); err != nil {
+		return errorf(CodeInternal, "format trailers: %w", err)
+	}
+	return m.Write(&envelope{
+		Data:  raw,
+		Flags: grpcFlagEnvelopeTrailer,
+	})
+}
+
+type grpcUnmarshaler struct {
+	envelopeReader envelopeReader
+	webTrailer     http.Header
+}
+
+func (u *grpcUnmarshaler) Unmarshal(message interface{}) *Error {
+	// delegate read packet and unmarshal processes to envelopeReader
+	err := u.envelopeReader.Unmarshal(message)
+	if err == nil {
+		return nil
+	}
+	if !errors.Is(err, errSpecialEnvelope) {
+		return err
+	}
+	// for special envelope
+	env := u.envelopeReader.last
+	// for non-web grpc, last envelope needs to set grpcFlagEnvelopeTrailer
+	if !env.IsSet(grpcFlagEnvelopeTrailer) {
+		return errorf(CodeInternal, "protocol error: invalid envelope flags %d", env.Flags)
+	}
+
+	// Per the gRPC-Web specification, trailers should be encoded as an HTTP/1
+	// headers block _without_ the terminating newline. To make the headers
+	// parseable by net/textproto, we need to add the newline.
+	if err := env.Data.WriteByte('\n'); err != nil {
+		return errorf(CodeInternal, "unmarshal web trailers: %w", err)
+	}
+	bufferedReader := bufio.NewReader(env.Data)
+	mimeReader := textproto.NewReader(bufferedReader)
+	mimeHeader, mimeErr := mimeReader.ReadMIMEHeader()
+	if mimeErr != nil {
+		return errorf(
+			CodeInternal,
+			"gRPC-Web protocol error: trailers invalid: %w",
+			mimeErr,
+		)
+	}
+	u.webTrailer = http.Header(mimeHeader)
+	return errSpecialEnvelope
+}
+
+func (u *grpcUnmarshaler) WebTrailer() http.Header {
+	return u.webTrailer
+}
+
+func grpcValidateResponse(
+	response *http.Response,
+	header, trailer http.Header,
+	availableCompressors readOnlyCompressionPools,
+	bufferPool *bufferPool,
+	protobuf Codec,
+) *Error {
+	if response.StatusCode != http.StatusOK {
+		return errorf(grpcHTTPToCode(response.StatusCode), "HTTP status %v", response.Status)
+	}
+	if compression := getHeaderCanonical(response.Header, grpcHeaderCompression); compression != "" &&
+		compression != compressionIdentity &&
+		!availableCompressors.Contains(compression) {
+		// Per https://github.com/grpc/grpc/blob/master/doc/compression.md, we
+		// should return CodeInternal and specify acceptable compression(s) (in
+		// addition to setting the Grpc-Accept-Encoding header).
+		return errorf(
+			CodeInternal,
+			"unknown encoding %q: accepted encodings are %v",
+			compression,
+			availableCompressors.CommaSeparatedNames(),
+		)
+	}
+	// When there's no body, gRPC and gRPC-Web servers may send error information
+	// in the HTTP headers.
+	if err := grpcErrorFromTrailer(
+		bufferPool,
+		protobuf,
+		response.Header,
+	); err != nil && !errors.Is(err, errTrailersWithoutGRPCStatus) {
+		// Per the specification, only the HTTP status code and Content-Type should
+		// be treated as headers. The rest should be treated as trailing metadata.
+		if contentType := getHeaderCanonical(response.Header, headerContentType); contentType != "" {
+			setHeaderCanonical(header, headerContentType, contentType)
+		}
+		mergeHeaders(trailer, response.Header)
+		delHeaderCanonical(trailer, headerContentType)
+		// Also set the error metadata
+		err.meta = header.Clone()
+		mergeHeaders(err.meta, trailer)
+		return err
+	}
+	// The response is valid, so we should expose the headers.
+	mergeHeaders(header, response.Header)
+	return nil
+}
+
+func grpcHTTPToCode(httpCode int) Code {
+	// https://github.com/grpc/grpc/blob/master/doc/http-grpc-status-mapping.md
+	// Note that this is not just the inverse of the gRPC-to-HTTP mapping.
+	switch httpCode {
+	case 400:
+		return CodeInternal
+	case 401:
+		return CodeUnauthenticated
+	case 403:
+		return CodePermissionDenied
+	case 404:
+		return CodeUnimplemented
+	case 429:
+		return CodeUnavailable
+	case 502, 503, 504:
+		return CodeUnavailable
+	default:
+		return CodeUnknown
+	}
+}
+
+// The gRPC wire protocol specifies that errors should be serialized using the
+// binary Protobuf format, even if the messages in the request/response stream
+// use a different codec. Consequently, this function needs a Protobuf codec to
+// unmarshal error information in the headers.
+func grpcErrorFromTrailer(bufferPool *bufferPool, protobuf Codec, trailer http.Header) *Error {
+	codeHeader := getHeaderCanonical(trailer, grpcHeaderStatus)
+	if codeHeader == "" {
+		return NewError(CodeInternal, errTrailersWithoutGRPCStatus)
+	}
+	if codeHeader == "0" {
+		return nil
+	}
+
+	code, err := strconv.ParseUint(codeHeader, 10 /* base */, 32 /* bitsize */)
+	if err != nil {
+		return errorf(CodeInternal, "gRPC protocol error: invalid error code %q", codeHeader)
+	}
+	message := grpcPercentDecode(bufferPool, getHeaderCanonical(trailer, grpcHeaderMessage))
+	retErr := NewWireError(Code(code), errors.New(message))
+
+	detailsBinaryEncoded := getHeaderCanonical(trailer, grpcHeaderDetails)
+	if len(detailsBinaryEncoded) > 0 {
+		detailsBinary, err := DecodeBinaryHeader(detailsBinaryEncoded)
+		if err != nil {
+			return errorf(CodeInternal, "server returned invalid grpc-status-details-bin trailer: %w", err)
+		}
+		var status statusv1.Status
+		if err := protobuf.Unmarshal(detailsBinary, &status); err != nil {
+			return errorf(CodeInternal, "server returned invalid protobuf for error details: %w", err)
+		}
+		for _, d := range status.Details {
+			retErr.details = append(retErr.details, &ErrorDetail{pb: d})
+		}
+		// Prefer the Protobuf-encoded data to the headers (grpc-go does this too).
+		retErr.code = Code(status.Code)
+		retErr.err = errors.New(status.Message)
+	}
+
+	return retErr
+}
+
+func grpcParseTimeout(timeout string) (time.Duration, error) {
+	if timeout == "" {
+		return 0, errNoTimeout
+	}
+	unit, ok := grpcTimeoutUnitLookup[timeout[len(timeout)-1]]
+	if !ok {
+		return 0, fmt.Errorf("gRPC protocol error: timeout %q has invalid unit", timeout)
+	}
+	num, err := strconv.ParseInt(timeout[:len(timeout)-1], 10 /* base */, 64 /* bitsize */)
+	if err != nil || num < 0 {
+		return 0, fmt.Errorf("gRPC protocol error: invalid timeout %q", timeout)
+	}
+	if num > 99999999 { // timeout must be ASCII string of at most 8 digits
+		return 0, fmt.Errorf("gRPC protocol error: timeout %q is too long", timeout)
+	}
+	if unit == time.Hour && num > grpcTimeoutMaxHours {
+		// Timeout is effectively unbounded, so ignore it. The grpc-go
+		// implementation does the same thing.
+		return 0, errNoTimeout
+	}
+	return time.Duration(num) * unit, nil
+}
+
+func grpcEncodeTimeout(timeout time.Duration) (string, error) {
+	if timeout <= 0 {
+		return "0n", nil
+	}
+	for _, pair := range grpcTimeoutUnits {
+		digits := strconv.FormatInt(int64(timeout/pair.size), 10 /* base */)
+		if len(digits) < grpcMaxTimeoutChars {
+			return digits + string(pair.char), nil
+		}
+	}
+	// The max time.Duration is smaller than the maximum expressible gRPC
+	// timeout, so we can't reach this case.
+	return "", errNoTimeout
+}
+
+func grpcCodecFromContentType(contentType string) string {
+	if contentType == grpcContentTypeDefault {
+		// implicitly protobuf
+		return codecNameProto
+	}
+
+	return strings.TrimPrefix(contentType, grpcContentTypePrefix)
+}
+
+func grpcContentTypeFromCodecName(name string) string {
+	return grpcContentTypePrefix + name
+}
+
+func grpcErrorToTrailer(bufferPool *bufferPool, trailer http.Header, protobuf Codec, err error) {
+	if err == nil {
+		setHeaderCanonical(trailer, grpcHeaderStatus, "0") // zero is the gRPC OK status
+		setHeaderCanonical(trailer, grpcHeaderMessage, "")
+		return
+	}
+	status := grpcStatusFromError(err)
+	code := strconv.Itoa(int(status.Code))
+	bin, binErr := protobuf.Marshal(status)
+	if binErr != nil {
+		setHeaderCanonical(
+			trailer,
+			grpcHeaderStatus,
+			strconv.FormatInt(int64(CodeInternal), 10 /* base */),
+		)
+		setHeaderCanonical(
+			trailer,
+			grpcHeaderMessage,
+			grpcPercentEncode(
+				bufferPool,
+				fmt.Sprintf("marshal protobuf status: %v", binErr),
+			),
+		)
+		return
+	}
+	if connectErr, ok := asError(err); ok {
+		mergeHeaders(trailer, connectErr.meta)
+	}
+	setHeaderCanonical(trailer, grpcHeaderStatus, code)
+	setHeaderCanonical(trailer, grpcHeaderMessage, grpcPercentEncode(bufferPool, status.Message))
+	setHeaderCanonical(trailer, grpcHeaderDetails, EncodeBinaryHeader(bin))
+}
+
+func grpcStatusFromError(err error) *statusv1.Status {
+	status := &statusv1.Status{
+		Code:    int32(CodeUnknown),
+		Message: err.Error(),
+	}
+	if connectErr, ok := asError(err); ok {
+		status.Code = int32(connectErr.Code())
+		status.Message = connectErr.Message()
+		status.Details = connectErr.detailsAsAny()
+	}
+	return status
+}
+
+// grpcPercentEncode follows RFC 3986 Section 2.1 and the gRPC HTTP/2 spec.
+// It's a variant of URL-encoding with fewer reserved characters. It's intended
+// to take UTF-8 encoded text and escape non-ASCII bytes so that they're valid
+// HTTP/1 headers, while still maximizing readability of the data on the wire.
+//
+// The grpc-message trailer (used for human-readable error messages) should be
+// percent-encoded.
+//
+// References:
+//
+//	https://github.com/grpc/grpc/blob/master/doc/PROTOCOL-HTTP2.md#responses
+//	https://datatracker.ietf.org/doc/html/rfc3986#section-2.1
+func grpcPercentEncode(bufferPool *bufferPool, msg string) string {
+	for i := 0; i < len(msg); i++ {
+		// Characters that need to be escaped are defined in gRPC's HTTP/2 spec.
+		// They're different from the generic set defined in RFC 3986.
+		if c := msg[i]; c < ' ' || c > '~' || c == '%' {
+			return grpcPercentEncodeSlow(bufferPool, msg, i)
+		}
+	}
+	return msg
+}
+
+// msg needs some percent-escaping. Bytes before offset don't require
+// percent-encoding, so they can be copied to the output as-is.
+func grpcPercentEncodeSlow(bufferPool *bufferPool, msg string, offset int) string {
+	out := bufferPool.Get()
+	defer bufferPool.Put(out)
+	out.WriteString(msg[:offset])
+	for i := offset; i < len(msg); i++ {
+		c := msg[i]
+		if c < ' ' || c > '~' || c == '%' {
+			out.WriteString(fmt.Sprintf("%%%02X", c))
+			continue
+		}
+		out.WriteByte(c)
+	}
+	return out.String()
+}
+
+func grpcPercentDecode(bufferPool *bufferPool, encoded string) string {
+	for i := 0; i < len(encoded); i++ {
+		if c := encoded[i]; c == '%' && i+2 < len(encoded) {
+			return grpcPercentDecodeSlow(bufferPool, encoded, i)
+		}
+	}
+	return encoded
+}
+
+// Similar to percentEncodeSlow: encoded is percent-encoded, and needs to be
+// decoded byte-by-byte starting at offset.
+func grpcPercentDecodeSlow(bufferPool *bufferPool, encoded string, offset int) string {
+	out := bufferPool.Get()
+	defer bufferPool.Put(out)
+	out.WriteString(encoded[:offset])
+	for i := offset; i < len(encoded); i++ {
+		c := encoded[i]
+		if c != '%' || i+2 >= len(encoded) {
+			out.WriteByte(c)
+			continue
+		}
+		parsed, err := strconv.ParseUint(encoded[i+1:i+3], 16 /* hex */, 8 /* bitsize */)
+		if err != nil {
+			out.WriteRune(utf8.RuneError)
+		} else {
+			out.WriteByte(byte(parsed))
+		}
+		i += 2
+	}
+	return out.String()
+}
diff --git a/protocol/triple/triple_protocol/protocol_grpc_test.go b/protocol/triple/triple_protocol/protocol_grpc_test.go
new file mode 100644
index 0000000..555db36
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol_grpc_test.go
@@ -0,0 +1,201 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"errors"
+	"math"
+	"net/http"
+	"net/http/httptest"
+	"strings"
+	"testing"
+	"testing/quick"
+	"time"
+	"unicode/utf8"
+)
+
+import (
+	"github.com/google/go-cmp/cmp"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestGRPCHandlerSender(t *testing.T) {
+	t.Parallel()
+	newConn := func(web bool) *grpcHandlerConn {
+		responseWriter := httptest.NewRecorder()
+		protobufCodec := &protoBinaryCodec{}
+		bufferPool := newBufferPool()
+		request, err := http.NewRequest(
+			http.MethodPost,
+			"https://demo.example.com",
+			strings.NewReader(""),
+		)
+		assert.Nil(t, err)
+		return &grpcHandlerConn{
+			spec:       Spec{},
+			web:        web,
+			bufferPool: bufferPool,
+			protobuf:   protobufCodec,
+			marshaler: grpcMarshaler{
+				envelopeWriter: envelopeWriter{
+					writer:     responseWriter,
+					codec:      protobufCodec,
+					bufferPool: bufferPool,
+				},
+			},
+			responseWriter:  responseWriter,
+			responseHeader:  make(http.Header),
+			responseTrailer: make(http.Header),
+			request:         request,
+			unmarshaler: grpcUnmarshaler{
+				envelopeReader: envelopeReader{
+					reader:     request.Body,
+					codec:      protobufCodec,
+					bufferPool: bufferPool,
+				},
+			},
+		}
+	}
+	t.Run("web", func(t *testing.T) {
+		t.Parallel()
+		testGRPCHandlerConnMetadata(t, newConn(true))
+	})
+	t.Run("http2", func(t *testing.T) {
+		t.Parallel()
+		testGRPCHandlerConnMetadata(t, newConn(false))
+	})
+}
+
+func testGRPCHandlerConnMetadata(t *testing.T, conn handlerConnCloser) {
+	// Closing the sender shouldn't unpredictably mutate user-visible headers or
+	// trailers.
+	t.Helper()
+	expectHeaders := conn.ResponseHeader().Clone()
+	expectTrailers := conn.ResponseTrailer().Clone()
+	conn.Close(NewError(CodeUnavailable, errors.New("oh no")))
+	if diff := cmp.Diff(expectHeaders, conn.ResponseHeader()); diff != "" {
+		t.Errorf("headers changed:\n%s", diff)
+	}
+	gotTrailers := conn.ResponseTrailer()
+	if diff := cmp.Diff(expectTrailers, gotTrailers); diff != "" {
+		t.Errorf("trailers changed:\n%s", diff)
+	}
+}
+
+func TestGRPCParseTimeout(t *testing.T) {
+	t.Parallel()
+	_, err := grpcParseTimeout("")
+	assert.True(t, errors.Is(err, errNoTimeout))
+
+	_, err = grpcParseTimeout("foo")
+	assert.NotNil(t, err)
+	_, err = grpcParseTimeout("12xS")
+	assert.NotNil(t, err)
+	_, err = grpcParseTimeout("999999999n") // 9 digits
+	assert.NotNil(t, err)
+	assert.False(t, errors.Is(err, errNoTimeout))
+	_, err = grpcParseTimeout("99999999H") // 8 digits but overflows time.Duration
+	assert.True(t, errors.Is(err, errNoTimeout))
+
+	duration, err := grpcParseTimeout("45S")
+	assert.Nil(t, err)
+	assert.Equal(t, duration, 45*time.Second)
+
+	const long = "99999999S"
+	duration, err = grpcParseTimeout(long) // 8 digits, shouldn't overflow
+	assert.Nil(t, err)
+	assert.Equal(t, duration, 99999999*time.Second)
+}
+
+func TestGRPCEncodeTimeout(t *testing.T) {
+	t.Parallel()
+	timeout, err := grpcEncodeTimeout(time.Hour + time.Second)
+	assert.Nil(t, err)
+	assert.Equal(t, timeout, "3601000m")
+	timeout, err = grpcEncodeTimeout(time.Duration(math.MaxInt64))
+	assert.Nil(t, err)
+	assert.Equal(t, timeout, "2562047H")
+	timeout, err = grpcEncodeTimeout(-1 * time.Hour)
+	assert.Nil(t, err)
+	assert.Equal(t, timeout, "0n")
+}
+
+func TestGRPCEncodeTimeoutQuick(t *testing.T) {
+	t.Parallel()
+	// Ensure that the error case is actually unreachable.
+	encode := func(d time.Duration) bool {
+		_, err := grpcEncodeTimeout(d)
+		return err == nil
+	}
+	if err := quick.Check(encode, nil); err != nil {
+		t.Error(err)
+	}
+}
+
+func TestGRPCPercentEncodingQuick(t *testing.T) {
+	t.Parallel()
+	pool := newBufferPool()
+	roundtrip := func(input string) bool {
+		if !utf8.ValidString(input) {
+			return true
+		}
+		encoded := grpcPercentEncode(pool, input)
+		decoded := grpcPercentDecode(pool, encoded)
+		return decoded == input
+	}
+	if err := quick.Check(roundtrip, nil /* config */); err != nil {
+		t.Error(err)
+	}
+}
+
+func TestGRPCPercentEncoding(t *testing.T) {
+	t.Parallel()
+	pool := newBufferPool()
+	roundtrip := func(input string) {
+		assert.True(t, utf8.ValidString(input), assert.Sprintf("input invalid UTF-8"))
+		encoded := grpcPercentEncode(pool, input)
+		t.Logf("%q encoded as %q", input, encoded)
+		decoded := grpcPercentDecode(pool, encoded)
+		assert.Equal(t, decoded, input)
+	}
+
+	roundtrip("foo")
+	roundtrip("foo bar")
+	roundtrip(`foo%bar`)
+	roundtrip("fiancée")
+}
+
+func TestGRPCWebTrailerMarshalling(t *testing.T) {
+	t.Parallel()
+	responseWriter := httptest.NewRecorder()
+	marshaler := grpcMarshaler{
+		envelopeWriter: envelopeWriter{
+			writer:     responseWriter,
+			bufferPool: newBufferPool(),
+		},
+	}
+	trailer := http.Header{}
+	trailer.Add("grpc-status", "0")
+	trailer.Add("Grpc-Message", "Foo")
+	trailer.Add("User-Provided", "bar")
+	err := marshaler.MarshalWebTrailers(trailer)
+	assert.Nil(t, err)
+	responseWriter.Body.Next(5) // skip flags and message length
+	marshalled := responseWriter.Body.String()
+	assert.Equal(t, marshalled, "grpc-message: Foo\r\ngrpc-status: 0\r\nuser-provided: bar\r\n")
+}
diff --git a/protocol/triple/triple_protocol/protocol_test.go b/protocol/triple/triple_protocol/protocol_test.go
new file mode 100644
index 0000000..ffbc09a
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol_test.go
@@ -0,0 +1,67 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"testing"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestCanonicalizeContentType(t *testing.T) {
+	t.Parallel()
+	tests := []struct {
+		name string
+		arg  string
+		want string
+	}{
+		{name: "uppercase should be normalized", arg: "APPLICATION/json", want: "application/json"},
+		{name: "charset param should be treated as lowercase", arg: "application/json; charset=UTF-8", want: "application/json; charset=utf-8"},
+		{name: "non charset param should not be changed", arg: "multipart/form-data; boundary=fooBar", want: "multipart/form-data; boundary=fooBar"},
+		{name: "no parameters should be normalized", arg: "APPLICATION/json;  ", want: "application/json"},
+	}
+	for _, tt := range tests {
+		tt := tt
+		t.Run(tt.name, func(t *testing.T) {
+			t.Parallel()
+			assert.Equal(t, canonicalizeContentType(tt.arg), tt.want)
+		})
+	}
+}
+
+func BenchmarkCanonicalizeContentType(b *testing.B) {
+	b.Run("simple", func(b *testing.B) {
+		for i := 0; i < b.N; i++ {
+			_ = canonicalizeContentType("application/json")
+		}
+		b.ReportAllocs()
+	})
+
+	b.Run("with charset", func(b *testing.B) {
+		for i := 0; i < b.N; i++ {
+			_ = canonicalizeContentType("application/json; charset=utf-8")
+		}
+		b.ReportAllocs()
+	})
+
+	b.Run("with other param", func(b *testing.B) {
+		for i := 0; i < b.N; i++ {
+			_ = canonicalizeContentType("application/json; foo=utf-8")
+		}
+		b.ReportAllocs()
+	})
+}
diff --git a/protocol/triple/triple_protocol/protocol_triple.go b/protocol/triple/triple_protocol/protocol_triple.go
new file mode 100644
index 0000000..cde9cf0
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol_triple.go
@@ -0,0 +1,766 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"bytes"
+	"context"
+	"encoding/base64"
+	"encoding/json"
+	"errors"
+	"fmt"
+	"io"
+	"math"
+	"net/http"
+	"runtime"
+	"strconv"
+	"strings"
+	"time"
+)
+
+import (
+	"google.golang.org/protobuf/types/known/anypb"
+)
+
+const (
+	tripleUnaryHeaderCompression       = "Content-Encoding"
+	tripleUnaryHeaderAcceptCompression = "Accept-Encoding"
+	tripleUnaryTrailerPrefix           = "Trailer-"
+	tripleHeaderTimeout                = "Triple-Timeout-Ms"
+	tripleHeaderProtocolVersion        = "Triple-Protocol-Version"
+	tripleProtocolVersion              = "1"
+
+	tripleUnaryContentTypePrefix = "application/"
+	tripleUnaryContentTypeJSON   = tripleUnaryContentTypePrefix + "json"
+)
+
+// defaultTripleUserAgent returns a User-Agent string similar to those used in gRPC.
+//
+//nolint:gochecknoglobals
+var defaultTripleUserAgent = fmt.Sprintf("triple-go/%s (%s)", Version, runtime.Version())
+
+type protocolTriple struct{}
+
+// NewHandler implements protocol, so it must return an interface.
+func (*protocolTriple) NewHandler(params *protocolHandlerParams) protocolHandler {
+	if params.Spec.StreamType != StreamTypeUnary {
+		panic("protocol triple does not support stream type rpc")
+	}
+	methods := make(map[string]struct{})
+	// we decide not to support GET
+	methods[http.MethodPost] = struct{}{}
+
+	contentTypes := make(map[string]struct{})
+	for _, name := range params.Codecs.Names() {
+		contentTypes[canonicalizeContentType(tripleUnaryContentTypePrefix+name)] = struct{}{}
+	}
+
+	return &tripleHandler{
+		protocolHandlerParams: *params,
+		methods:               methods,
+		accept:                contentTypes,
+	}
+}
+
+// NewClient implements protocol, so it must return an interface.
+func (*protocolTriple) NewClient(params *protocolClientParams) (protocolClient, error) {
+	return &tripleClient{
+		protocolClientParams: *params,
+		peer:                 newPeerFromURL(params.URL, ProtocolTriple),
+	}, nil
+}
+
+type tripleHandler struct {
+	protocolHandlerParams
+
+	methods map[string]struct{}
+	accept  map[string]struct{}
+}
+
+func (h *tripleHandler) Methods() map[string]struct{} {
+	return h.methods
+}
+
+func (h *tripleHandler) ContentTypes() map[string]struct{} {
+	return h.accept
+}
+
+func (*tripleHandler) SetTimeout(request *http.Request) (context.Context, context.CancelFunc, error) {
+	timeout := getHeaderCanonical(request.Header, tripleHeaderTimeout)
+	if timeout == "" {
+		return request.Context(), nil, nil
+	}
+	if len(timeout) > 10 {
+		return nil, nil, errorf(CodeInvalidArgument, "parse timeout: %q has >10 digits", timeout)
+	}
+	millis, err := strconv.ParseInt(timeout, 10 /* base */, 64 /* bitsize */)
+	if err != nil {
+		return nil, nil, errorf(CodeInvalidArgument, "parse timeout: %w", err)
+	}
+	ctx, cancel := context.WithTimeout(
+		request.Context(),
+		time.Duration(millis)*time.Millisecond,
+	)
+	return ctx, cancel, nil
+}
+
+func (h *tripleHandler) CanHandlePayload(request *http.Request, contentType string) bool {
+	_, ok := h.accept[contentType]
+	return ok
+}
+
+func (h *tripleHandler) NewConn(
+	responseWriter http.ResponseWriter,
+	request *http.Request,
+) (handlerConnCloser, bool) {
+	if request.Method != http.MethodPost {
+		return nil, false
+	}
+	// We need to parse metadata before entering the interceptor stack; we'll
+	// send the error to the client later on.
+	var contentEncoding, acceptEncoding string
+	contentEncoding = getHeaderCanonical(request.Header, tripleUnaryHeaderCompression)
+	acceptEncoding = getHeaderCanonical(request.Header, tripleUnaryHeaderAcceptCompression)
+	requestCompression, responseCompression, failed := negotiateCompression(
+		h.CompressionPools,
+		contentEncoding,
+		acceptEncoding,
+	)
+	if failed == nil {
+		version := getHeaderCanonical(request.Header, tripleHeaderProtocolVersion)
+		if version == "" && h.RequireTripleProtocolHeader {
+			failed = errorf(CodeInvalidArgument, "missing required header: set %s to %q", tripleHeaderProtocolVersion, tripleProtocolVersion)
+		} else if version != "" && version != tripleProtocolVersion {
+			failed = errorf(CodeInvalidArgument, "%s must be %q: got %q", tripleHeaderProtocolVersion, tripleProtocolVersion, version)
+		}
+	}
+
+	var requestBody io.ReadCloser
+	var contentType, codecName string
+	requestBody = request.Body
+	contentType = getHeaderCanonical(request.Header, headerContentType)
+	codecName = tripleCodecFromContentType(
+		h.Spec.StreamType,
+		contentType,
+	)
+
+	codec := h.Codecs.Get(codecName)
+	// todo:// need to figure it out
+	// The codec can be nil in the GET request case; that's okay: when failed
+	// is non-nil, codec is never used.
+	if failed == nil && codec == nil {
+		failed = errorf(CodeInvalidArgument, "invalid message encoding: %q", codecName)
+	}
+
+	// Write any remaining headers here:
+	// (1) any writes to the stream will implicitly send the headers, so we
+	// should get all of gRPC's required response headers ready.
+	// (2) interceptors should be able to see these headers.
+	//
+	// Since we know that these header keys are already in canonical form, we can
+	// skip the normalization in Header.Set.
+	header := responseWriter.Header()
+	header[headerContentType] = []string{contentType}
+	acceptCompressionHeader := tripleUnaryHeaderAcceptCompression
+	header[acceptCompressionHeader] = []string{h.CompressionPools.CommaSeparatedNames()}
+
+	var conn handlerConnCloser
+	peer := Peer{
+		Addr:     request.RemoteAddr,
+		Protocol: ProtocolTriple,
+	}
+	conn = &tripleUnaryHandlerConn{
+		spec:           h.Spec,
+		peer:           peer,
+		request:        request,
+		responseWriter: responseWriter,
+		marshaler: tripleUnaryMarshaler{
+			writer:           responseWriter,
+			codec:            codec,
+			compressMinBytes: h.CompressMinBytes,
+			compressionName:  responseCompression,
+			compressionPool:  h.CompressionPools.Get(responseCompression),
+			bufferPool:       h.BufferPool,
+			header:           responseWriter.Header(),
+			sendMaxBytes:     h.SendMaxBytes,
+		},
+		unmarshaler: tripleUnaryUnmarshaler{
+			reader:          requestBody,
+			codec:           codec,
+			compressionPool: h.CompressionPools.Get(requestCompression),
+			bufferPool:      h.BufferPool,
+			readMaxBytes:    h.ReadMaxBytes,
+		},
+		responseTrailer: make(http.Header),
+	}
+	conn = wrapHandlerConnWithCodedErrors(conn)
+
+	if failed != nil {
+		// Negotiation failed, so we can't establish a stream.
+		_ = conn.Close(failed)
+		return nil, false
+	}
+	return conn, true
+}
+
+type tripleClient struct {
+	protocolClientParams
+
+	peer Peer
+}
+
+func (c *tripleClient) Peer() Peer {
+	return c.peer
+}
+
+func (c *tripleClient) WriteRequestHeader(streamType StreamType, header http.Header) {
+	// We know these header keys are in canonical form, so we can bypass all the
+	// checks in Header.Set.
+	if getHeaderCanonical(header, headerUserAgent) == "" {
+		header[headerUserAgent] = []string{defaultTripleUserAgent}
+	}
+	header[tripleHeaderProtocolVersion] = []string{tripleProtocolVersion}
+	header[headerContentType] = []string{
+		tripleContentTypeFromCodecName(streamType, c.Codec.Name()),
+	}
+	acceptCompressionHeader := tripleUnaryHeaderAcceptCompression
+	if acceptCompression := c.CompressionPools.CommaSeparatedNames(); acceptCompression != "" {
+		header[acceptCompressionHeader] = []string{acceptCompression}
+	}
+}
+
+func (c *tripleClient) NewConn(
+	ctx context.Context,
+	spec Spec,
+	header http.Header,
+) StreamingClientConn {
+	if deadline, ok := ctx.Deadline(); ok {
+		millis := int64(time.Until(deadline) / time.Millisecond)
+		if millis > 0 {
+			encoded := strconv.FormatInt(millis, 10 /* base */)
+			if len(encoded) <= 10 {
+				header[tripleHeaderTimeout] = []string{encoded}
+			} // else effectively unbounded
+		}
+	}
+	duplexCall := newDuplexHTTPCall(ctx, c.HTTPClient, c.URL, spec, header)
+	var conn StreamingClientConn
+	unaryConn := &tripleUnaryClientConn{
+		spec:             spec,
+		peer:             c.Peer(),
+		duplexCall:       duplexCall,
+		compressionPools: c.CompressionPools,
+		bufferPool:       c.BufferPool,
+		marshaler: tripleUnaryRequestMarshaler{
+			tripleUnaryMarshaler: tripleUnaryMarshaler{
+				writer:           duplexCall,
+				codec:            c.Codec,
+				compressMinBytes: c.CompressMinBytes,
+				compressionName:  c.CompressionName,
+				compressionPool:  c.CompressionPools.Get(c.CompressionName),
+				bufferPool:       c.BufferPool,
+				header:           duplexCall.Header(),
+				sendMaxBytes:     c.SendMaxBytes,
+			},
+		},
+		unmarshaler: tripleUnaryUnmarshaler{
+			reader:       duplexCall,
+			codec:        c.Codec,
+			bufferPool:   c.BufferPool,
+			readMaxBytes: c.ReadMaxBytes,
+		},
+		responseHeader:  make(http.Header),
+		responseTrailer: make(http.Header),
+	}
+	conn = unaryConn
+	duplexCall.SetValidateResponse(unaryConn.validateResponse)
+	return wrapClientConnWithCodedErrors(conn)
+}
+
+type tripleUnaryClientConn struct {
+	spec             Spec
+	peer             Peer
+	duplexCall       *duplexHTTPCall
+	compressionPools readOnlyCompressionPools
+	bufferPool       *bufferPool
+	marshaler        tripleUnaryRequestMarshaler
+	unmarshaler      tripleUnaryUnmarshaler
+	responseHeader   http.Header
+	responseTrailer  http.Header
+}
+
+func (cc *tripleUnaryClientConn) Spec() Spec {
+	return cc.spec
+}
+
+func (cc *tripleUnaryClientConn) Peer() Peer {
+	return cc.peer
+}
+
+func (cc *tripleUnaryClientConn) Send(msg interface{}) error {
+	if err := cc.marshaler.Marshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (cc *tripleUnaryClientConn) RequestHeader() http.Header {
+	return cc.duplexCall.Header()
+}
+
+func (cc *tripleUnaryClientConn) CloseRequest() error {
+	return cc.duplexCall.CloseWrite()
+}
+
+func (cc *tripleUnaryClientConn) Receive(msg interface{}) error {
+	cc.duplexCall.BlockUntilResponseReady()
+	if err := cc.unmarshaler.Unmarshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (cc *tripleUnaryClientConn) ResponseHeader() http.Header {
+	cc.duplexCall.BlockUntilResponseReady()
+	return cc.responseHeader
+}
+
+func (cc *tripleUnaryClientConn) ResponseTrailer() http.Header {
+	cc.duplexCall.BlockUntilResponseReady()
+	return cc.responseTrailer
+}
+
+func (cc *tripleUnaryClientConn) CloseResponse() error {
+	return cc.duplexCall.CloseRead()
+}
+
+func (cc *tripleUnaryClientConn) validateResponse(response *http.Response) *Error {
+	for k, v := range response.Header {
+		if !strings.HasPrefix(k, tripleUnaryTrailerPrefix) {
+			cc.responseHeader[k] = v
+			continue
+		}
+		cc.responseTrailer[strings.TrimPrefix(k, tripleUnaryTrailerPrefix)] = v
+	}
+	compression := getHeaderCanonical(response.Header, tripleUnaryHeaderCompression)
+	if compression != "" &&
+		compression != compressionIdentity &&
+		!cc.compressionPools.Contains(compression) {
+		return errorf(
+			CodeInternal,
+			"unknown encoding %q: accepted encodings are %v",
+			compression,
+			cc.compressionPools.CommaSeparatedNames(),
+		)
+	}
+	if response.StatusCode != http.StatusOK {
+		unmarshaler := tripleUnaryUnmarshaler{
+			reader:          response.Body,
+			compressionPool: cc.compressionPools.Get(compression),
+			bufferPool:      cc.bufferPool,
+		}
+		var wireErr tripleWireError
+		if err := unmarshaler.UnmarshalFunc(&wireErr, json.Unmarshal); err != nil {
+			return NewError(
+				tripleHTTPToCode(response.StatusCode),
+				errors.New(response.Status),
+			)
+		}
+		serverErr := wireErr.asError()
+		if serverErr == nil {
+			return nil
+		}
+		serverErr.meta = cc.responseHeader.Clone()
+		mergeHeaders(serverErr.meta, cc.responseTrailer)
+		return serverErr
+	}
+	cc.unmarshaler.compressionPool = cc.compressionPools.Get(compression)
+	return nil
+}
+
+type tripleUnaryHandlerConn struct {
+	spec            Spec
+	peer            Peer
+	request         *http.Request
+	responseWriter  http.ResponseWriter
+	marshaler       tripleUnaryMarshaler
+	unmarshaler     tripleUnaryUnmarshaler
+	responseTrailer http.Header
+	wroteBody       bool
+}
+
+func (hc *tripleUnaryHandlerConn) Spec() Spec {
+	return hc.spec
+}
+
+func (hc *tripleUnaryHandlerConn) Peer() Peer {
+	return hc.peer
+}
+
+func (hc *tripleUnaryHandlerConn) Receive(msg interface{}) error {
+	if err := hc.unmarshaler.Unmarshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (hc *tripleUnaryHandlerConn) RequestHeader() http.Header {
+	return hc.request.Header
+}
+
+func (hc *tripleUnaryHandlerConn) Send(msg interface{}) error {
+	hc.wroteBody = true
+	hc.writeResponseHeader(nil /* error */)
+	if err := hc.marshaler.Marshal(msg); err != nil {
+		return err
+	}
+	return nil // must be a literal nil: nil *Error is a non-nil error
+}
+
+func (hc *tripleUnaryHandlerConn) ResponseHeader() http.Header {
+	return hc.responseWriter.Header()
+}
+
+func (hc *tripleUnaryHandlerConn) ResponseTrailer() http.Header {
+	return hc.responseTrailer
+}
+
+func (hc *tripleUnaryHandlerConn) Close(err error) error {
+	if !hc.wroteBody {
+		hc.writeResponseHeader(err)
+	}
+	if err == nil {
+		return hc.request.Body.Close()
+	}
+	// In unary triple, errors always use application/json.
+	setHeaderCanonical(hc.responseWriter.Header(), headerContentType, tripleUnaryContentTypeJSON)
+	hc.responseWriter.WriteHeader(tripleCodeToHTTP(CodeOf(err)))
+	data, marshalErr := json.Marshal(newTripleWireError(err))
+	if marshalErr != nil {
+		_ = hc.request.Body.Close()
+		return errorf(CodeInternal, "marshal error: %w", err)
+	}
+	if _, writeErr := hc.responseWriter.Write(data); writeErr != nil {
+		_ = hc.request.Body.Close()
+		return writeErr
+	}
+	return hc.request.Body.Close()
+}
+
+func (hc *tripleUnaryHandlerConn) writeResponseHeader(err error) {
+	header := hc.responseWriter.Header()
+	if err != nil {
+		if tripleErr, ok := asError(err); ok {
+			mergeHeaders(header, tripleErr.meta)
+		}
+	}
+	for k, v := range hc.responseTrailer {
+		header[tripleUnaryTrailerPrefix+k] = v
+	}
+}
+
+type tripleUnaryMarshaler struct {
+	writer           io.Writer
+	codec            Codec
+	compressMinBytes int
+	compressionName  string
+	compressionPool  *compressionPool
+	bufferPool       *bufferPool
+	header           http.Header
+	sendMaxBytes     int
+}
+
+func (m *tripleUnaryMarshaler) Marshal(message interface{}) *Error {
+	if message == nil {
+		return m.write(nil)
+	}
+	data, err := m.codec.Marshal(message)
+	if err != nil {
+		return errorf(CodeInternal, "marshal message: %w", err)
+	}
+	// Can't avoid allocating the slice, but we can reuse it.
+	uncompressed := bytes.NewBuffer(data)
+	defer m.bufferPool.Put(uncompressed)
+	if len(data) < m.compressMinBytes || m.compressionPool == nil {
+		if m.sendMaxBytes > 0 && len(data) > m.sendMaxBytes {
+			return NewError(CodeResourceExhausted, fmt.Errorf("message size %d exceeds sendMaxBytes %d", len(data), m.sendMaxBytes))
+		}
+		return m.write(data)
+	}
+	compressed := m.bufferPool.Get()
+	defer m.bufferPool.Put(compressed)
+	if err := m.compressionPool.Compress(compressed, uncompressed); err != nil {
+		return err
+	}
+	if m.sendMaxBytes > 0 && compressed.Len() > m.sendMaxBytes {
+		return NewError(CodeResourceExhausted, fmt.Errorf("compressed message size %d exceeds sendMaxBytes %d", compressed.Len(), m.sendMaxBytes))
+	}
+	setHeaderCanonical(m.header, tripleUnaryHeaderCompression, m.compressionName)
+	return m.write(compressed.Bytes())
+}
+
+func (m *tripleUnaryMarshaler) write(data []byte) *Error {
+	if _, err := m.writer.Write(data); err != nil {
+		if tripleErr, ok := asError(err); ok {
+			return tripleErr
+		}
+		return errorf(CodeUnknown, "write message: %w", err)
+	}
+	return nil
+}
+
+type tripleUnaryRequestMarshaler struct {
+	tripleUnaryMarshaler
+
+	stableCodec stableCodec
+	duplexCall  *duplexHTTPCall
+}
+
+func (m *tripleUnaryRequestMarshaler) Marshal(message interface{}) *Error {
+	return m.tripleUnaryMarshaler.Marshal(message)
+}
+
+type tripleUnaryUnmarshaler struct {
+	reader          io.Reader
+	codec           Codec
+	compressionPool *compressionPool
+	bufferPool      *bufferPool
+	alreadyRead     bool
+	readMaxBytes    int
+}
+
+func (u *tripleUnaryUnmarshaler) Unmarshal(message interface{}) *Error {
+	return u.UnmarshalFunc(message, u.codec.Unmarshal)
+}
+
+func (u *tripleUnaryUnmarshaler) UnmarshalFunc(message interface{}, unmarshal func([]byte, interface{}) error) *Error {
+	if u.alreadyRead {
+		return NewError(CodeInternal, io.EOF)
+	}
+	u.alreadyRead = true
+	data := u.bufferPool.Get()
+	defer u.bufferPool.Put(data)
+	reader := u.reader
+	if u.readMaxBytes > 0 && int64(u.readMaxBytes) < math.MaxInt64 {
+		reader = io.LimitReader(u.reader, int64(u.readMaxBytes)+1)
+	}
+	// ReadFrom ignores io.EOF, so any error here is real.
+	bytesRead, err := data.ReadFrom(reader)
+	if err != nil {
+		if tripleErr, ok := asError(err); ok {
+			return tripleErr
+		}
+		if readMaxBytesErr := asMaxBytesError(err, "read first %d bytes of message", bytesRead); readMaxBytesErr != nil {
+			return readMaxBytesErr
+		}
+		return errorf(CodeUnknown, "read message: %w", err)
+	}
+	if u.readMaxBytes > 0 && bytesRead > int64(u.readMaxBytes) {
+		// Attempt to read to end in order to allow tripleion re-use
+		discardedBytes, err := io.Copy(io.Discard, u.reader)
+		if err != nil {
+			return errorf(CodeResourceExhausted, "message is larger than configured max %d - unable to determine message size: %w", u.readMaxBytes, err)
+		}
+		return errorf(CodeResourceExhausted, "message size %d is larger than configured max %d", bytesRead+discardedBytes, u.readMaxBytes)
+	}
+	if data.Len() > 0 && u.compressionPool != nil {
+		decompressed := u.bufferPool.Get()
+		defer u.bufferPool.Put(decompressed)
+		if err := u.compressionPool.Decompress(decompressed, data, int64(u.readMaxBytes)); err != nil {
+			return err
+		}
+		data = decompressed
+	}
+	if err := unmarshal(data.Bytes(), message); err != nil {
+		return errorf(CodeInvalidArgument, "unmarshal into %T: %w", message, err)
+	}
+	return nil
+}
+
+type tripleWireDetail ErrorDetail
+
+func (d *tripleWireDetail) MarshalJSON() ([]byte, error) {
+	if d.wireJSON != "" {
+		// If we unmarshaled this detail from JSON, return the original data. This
+		// lets proxies w/o protobuf descriptors preserve human-readable details.
+		return []byte(d.wireJSON), nil
+	}
+	wire := struct {
+		Type  string          `json:"type"`
+		Value string          `json:"value"`
+		Debug json.RawMessage `json:"debug,omitempty"`
+	}{
+		Type:  strings.TrimPrefix(d.pb.TypeUrl, defaultAnyResolverPrefix),
+		Value: base64.RawStdEncoding.EncodeToString(d.pb.Value),
+	}
+	// Try to produce debug info, but expect failure when we don't have
+	// descriptors.
+	var codec protoJSONCodec
+	debug, err := codec.Marshal(d.pb)
+	if err == nil && len(debug) > 2 { // don't bother sending `{}`
+		wire.Debug = json.RawMessage(debug)
+	}
+	return json.Marshal(wire)
+}
+
+func (d *tripleWireDetail) UnmarshalJSON(data []byte) error {
+	var wire struct {
+		Type  string `json:"type"`
+		Value string `json:"value"`
+	}
+	if err := json.Unmarshal(data, &wire); err != nil {
+		return err
+	}
+	if !strings.Contains(wire.Type, "/") {
+		wire.Type = defaultAnyResolverPrefix + wire.Type
+	}
+	decoded, err := DecodeBinaryHeader(wire.Value)
+	if err != nil {
+		return fmt.Errorf("decode base64: %w", err)
+	}
+	*d = tripleWireDetail{
+		pb: &anypb.Any{
+			TypeUrl: wire.Type,
+			Value:   decoded,
+		},
+		wireJSON: string(data),
+	}
+	return nil
+}
+
+type tripleWireError struct {
+	Code    Code                `json:"code"`
+	Message string              `json:"message,omitempty"`
+	Details []*tripleWireDetail `json:"details,omitempty"`
+}
+
+func newTripleWireError(err error) *tripleWireError {
+	wire := &tripleWireError{
+		Code:    CodeUnknown,
+		Message: err.Error(),
+	}
+	if tripleErr, ok := asError(err); ok {
+		wire.Code = tripleErr.Code()
+		wire.Message = tripleErr.Message()
+		if len(tripleErr.details) > 0 {
+			wire.Details = make([]*tripleWireDetail, len(tripleErr.details))
+			for i, detail := range tripleErr.details {
+				wire.Details[i] = (*tripleWireDetail)(detail)
+			}
+		}
+	}
+	return wire
+}
+
+func (e *tripleWireError) asError() *Error {
+	if e == nil {
+		return nil
+	}
+	if e.Code < minCode || e.Code > maxCode {
+		e.Code = CodeUnknown
+	}
+	err := NewWireError(e.Code, errors.New(e.Message))
+	if len(e.Details) > 0 {
+		err.details = make([]*ErrorDetail, len(e.Details))
+		for i, detail := range e.Details {
+			err.details[i] = (*ErrorDetail)(detail)
+		}
+	}
+	return err
+}
+
+func tripleCodeToHTTP(code Code) int {
+	// Return literals rather than named constants from the HTTP package to make
+	// it easier to compare this function to the triple specification.
+	switch code {
+	case CodeCanceled:
+		return 408
+	case CodeUnknown:
+		return 500
+	case CodeInvalidArgument:
+		return 400
+	case CodeDeadlineExceeded:
+		return 408
+	case CodeNotFound:
+		return 404
+	case CodeAlreadyExists:
+		return 409
+	case CodePermissionDenied:
+		return 403
+	case CodeResourceExhausted:
+		return 429
+	case CodeFailedPrecondition:
+		return 412
+	case CodeAborted:
+		return 409
+	case CodeOutOfRange:
+		return 400
+	case CodeUnimplemented:
+		return 404
+	case CodeInternal:
+		return 500
+	case CodeUnavailable:
+		return 503
+	case CodeDataLoss:
+		return 500
+	case CodeUnauthenticated:
+		return 401
+	default:
+		return 500 // same as CodeUnknown
+	}
+}
+
+func tripleHTTPToCode(httpCode int) Code {
+	// As above, literals are easier to compare to the specification (vs named
+	// constants).
+	switch httpCode {
+	case 400:
+		return CodeInvalidArgument
+	case 401:
+		return CodeUnauthenticated
+	case 403:
+		return CodePermissionDenied
+	case 404:
+		return CodeUnimplemented
+	case 408:
+		return CodeDeadlineExceeded
+	case 412:
+		return CodeFailedPrecondition
+	case 413:
+		return CodeResourceExhausted
+	case 429:
+		return CodeUnavailable
+	case 431:
+		return CodeResourceExhausted
+	case 502, 503, 504:
+		return CodeUnavailable
+	default:
+		return CodeUnknown
+	}
+}
+
+func tripleCodecFromContentType(streamType StreamType, contentType string) string {
+	if streamType == StreamTypeUnary {
+		return strings.TrimPrefix(contentType, tripleUnaryContentTypePrefix)
+	}
+	return ""
+}
+
+func tripleContentTypeFromCodecName(streamType StreamType, name string) string {
+	if streamType == StreamTypeUnary {
+		return tripleUnaryContentTypePrefix + name
+	}
+	return ""
+}
diff --git a/protocol/triple/triple_protocol/protocol_triple_test.go b/protocol/triple/triple_protocol/protocol_triple_test.go
new file mode 100644
index 0000000..e3ff5d9
--- /dev/null
+++ b/protocol/triple/triple_protocol/protocol_triple_test.go
@@ -0,0 +1,61 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"encoding/json"
+	"strings"
+	"testing"
+	"time"
+)
+
+import (
+	"google.golang.org/protobuf/types/known/durationpb"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+)
+
+func TestTripleErrorDetailMarshaling(t *testing.T) {
+	t.Parallel()
+	detail, err := NewErrorDetail(durationpb.New(time.Second))
+	assert.Nil(t, err)
+	data, err := json.Marshal((*tripleWireDetail)(detail))
+	assert.Nil(t, err)
+	t.Logf("marshaled error detail: %s", string(data))
+
+	var unmarshaled tripleWireDetail
+	assert.Nil(t, json.Unmarshal(data, &unmarshaled))
+	assert.Equal(t, unmarshaled.wireJSON, string(data))
+	assert.Equal(t, unmarshaled.pb, detail.pb)
+}
+
+func TestTripleErrorDetailMarshalingNoDescriptor(t *testing.T) {
+	t.Parallel()
+	raw := `{"type":"acme.user.v1.User","value":"DEADBF",` +
+		`"debug":{"@type":"acme.user.v1.User","email":"someone@triple.build"}}`
+	var detail tripleWireDetail
+	assert.Nil(t, json.Unmarshal([]byte(raw), &detail))
+	assert.Equal(t, detail.pb.TypeUrl, defaultAnyResolverPrefix+"acme.user.v1.User")
+
+	_, err := (*ErrorDetail)(&detail).Value()
+	assert.NotNil(t, err)
+	assert.True(t, strings.HasSuffix(err.Error(), "not found"))
+
+	encoded, err := json.Marshal(&detail)
+	assert.Nil(t, err)
+	assert.Equal(t, string(encoded), raw)
+}
diff --git a/protocol/triple/triple_protocol/recover.go b/protocol/triple/triple_protocol/recover.go
new file mode 100644
index 0000000..f07c5d2
--- /dev/null
+++ b/protocol/triple/triple_protocol/recover.go
@@ -0,0 +1,99 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol
+
+import (
+	"context"
+	"net/http"
+)
+
+// recoverHandlerInterceptor lets handlers trap panics, perform side effects
+// (like emitting logs or metrics), and present a friendlier error message to
+// clients.
+//
+// This interceptor uses a somewhat unusual strategy to recover from panics.
+// The standard recovery idiom:
+//
+//	if r := recover(); r != nil { ... }
+//
+// isn't robust in the face of user error, because it doesn't handle
+// panic(nil). This occasionally happens by mistake, and it's a beast to debug
+// without a more robust idiom. See https://github.com/golang/go/issues/25448
+// for details.
+type recoverHandlerInterceptor struct {
+	Interceptor
+
+	handle func(context.Context, Spec, http.Header, interface{}) error
+}
+
+func (i *recoverHandlerInterceptor) WrapUnary(next UnaryFunc) UnaryFunc {
+	return func(ctx context.Context, req AnyRequest, resp AnyResponse) (retErr error) {
+		if req.Spec().IsClient {
+			return next(ctx, req, resp)
+		}
+		panicked := true
+		defer func() {
+			if panicked {
+				r := recover()
+				// net/http checks for ErrAbortHandler with ==, so we should too.
+				if r == http.ErrAbortHandler { //nolint:errorlint,goerr113
+					panic(r) //nolint:forbidigo
+				}
+				retErr = i.handle(ctx, req.Spec(), req.Header(), r)
+			}
+		}()
+		err := next(ctx, req, resp)
+		panicked = false
+		return err
+	}
+}
+
+func (i *recoverHandlerInterceptor) WrapUnaryHandler(next UnaryHandlerFunc) UnaryHandlerFunc {
+	return func(ctx context.Context, request AnyRequest) (resp AnyResponse, retErr error) {
+		panicked := true
+		defer func() {
+			if panicked {
+				r := recover()
+				// net/http checks for ErrAbortHandler with ==, so we should too.
+				if r == http.ErrAbortHandler { //nolint:errorlint,goerr113
+					panic(r) //nolint:forbidigo
+				}
+				retErr = i.handle(ctx, request.Spec(), request.Header(), r)
+			}
+		}()
+		response, retErr := next(ctx, request)
+		panicked = false
+		return response, retErr
+	}
+}
+
+func (i *recoverHandlerInterceptor) WrapStreamingHandler(next StreamingHandlerFunc) StreamingHandlerFunc {
+	return func(ctx context.Context, conn StreamingHandlerConn) (retErr error) {
+		panicked := true
+		defer func() {
+			if panicked {
+				r := recover()
+				// net/http checks for ErrAbortHandler with ==, so we should too.
+				if r == http.ErrAbortHandler { //nolint:errorlint,goerr113
+					panic(r) //nolint:forbidigo
+				}
+				retErr = i.handle(ctx, Spec{}, nil, r)
+			}
+		}()
+		err := next(ctx, conn)
+		panicked = false
+		return err
+	}
+}
diff --git a/protocol/triple/triple_protocol/recover_ext_test.go b/protocol/triple/triple_protocol/recover_ext_test.go
new file mode 100644
index 0000000..435d06b
--- /dev/null
+++ b/protocol/triple/triple_protocol/recover_ext_test.go
@@ -0,0 +1,109 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+package triple_protocol_test
+
+import (
+	"context"
+	"fmt"
+	"net/http"
+	"net/http/httptest"
+	"testing"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	pingv1triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+type panicPingServer struct {
+	pingv1triple.UnimplementedPingServiceHandler
+	panicWith interface{}
+}
+
+func (s *panicPingServer) Ping(
+	context.Context,
+	*triple_protocol.Request,
+) (*triple_protocol.Response, error) {
+	panic(s.panicWith) //nolint:forbidigo
+}
+
+func (s *panicPingServer) CountUp(
+	_ context.Context,
+	_ *triple_protocol.Request,
+	stream *triple_protocol.ServerStream,
+) error {
+	if err := stream.Send(&pingv1.CountUpResponse{}); err != nil {
+		return err
+	}
+	panic(s.panicWith) //nolint:forbidigo
+}
+
+func TestWithRecover(t *testing.T) {
+	t.Parallel()
+	handle := func(_ context.Context, _ triple_protocol.Spec, _ http.Header, r interface{}) error {
+		return triple_protocol.NewError(triple_protocol.CodeFailedPrecondition, fmt.Errorf("panic: %v", r))
+	}
+	assertHandled := func(err error) {
+		t.Helper()
+		assert.NotNil(t, err)
+		assert.Equal(t, triple_protocol.CodeOf(err), triple_protocol.CodeFailedPrecondition)
+	}
+	assertNotHandled := func(err error) {
+		t.Helper()
+		// When HTTP/2 handlers panic, net/http sends an RST_STREAM frame with code
+		// INTERNAL_ERROR. We should be mapping this back to CodeInternal.
+		assert.Equal(t, triple_protocol.CodeOf(err), triple_protocol.CodeInternal)
+	}
+	drainStream := func(stream *triple_protocol.ServerStreamForClient) error {
+		t.Helper()
+		defer stream.Close()
+		assert.True(t, stream.Receive(&pingv1.CountUpResponse{}))  // expect one response msg
+		assert.False(t, stream.Receive(&pingv1.CountUpResponse{})) // expect panic before second response msg
+		return stream.Err()
+	}
+	pinger := &panicPingServer{}
+	mux := http.NewServeMux()
+	mux.Handle(pingv1triple.NewPingServiceHandler(pinger, triple_protocol.WithRecover(handle)))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	defer server.Close()
+	client := pingv1triple.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+	)
+
+	for _, panicWith := range []interface{}{42, nil} {
+		pinger.panicWith = panicWith
+
+		err := client.Ping(context.Background(), triple_protocol.NewRequest(&pingv1.PingRequest{}), triple_protocol.NewResponse(&pingv1.PingResponse{}))
+		assertHandled(err)
+
+		stream, err := client.CountUp(context.Background(), triple_protocol.NewRequest(&pingv1.CountUpRequest{}))
+		assert.Nil(t, err)
+		assertHandled(drainStream(stream))
+	}
+
+	pinger.panicWith = http.ErrAbortHandler
+
+	err := client.Ping(context.Background(), triple_protocol.NewRequest(&pingv1.PingRequest{}), triple_protocol.NewResponse(&pingv1.PingResponse{}))
+	assertNotHandled(err)
+
+	stream, err := client.CountUp(context.Background(), triple_protocol.NewRequest(&pingv1.CountUpRequest{}))
+	assert.Nil(t, err)
+	assertNotHandled(drainStream(stream))
+}
diff --git a/protocol/triple/triple_protocol/triple.go b/protocol/triple/triple_protocol/triple.go
new file mode 100644
index 0000000..3fe0a7f
--- /dev/null
+++ b/protocol/triple/triple_protocol/triple.go
@@ -0,0 +1,351 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//      http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+
+// Package triple is a slim RPC framework built on Protocol Buffers and
+// [net/http]. In addition to supporting its own protocol, Connect handlers and
+// clients are wire-compatible with gRPC and gRPC-Web, including streaming.
+//
+// This documentation is intended to explain each type and function in
+// isolation. Walkthroughs, FAQs, and other narrative docs are available on the
+// [Connect website], and there's a working [demonstration service] on Github.
+//
+// [Connect website]: https://connect.build
+// [demonstration service]: https://github.com/bufbuild/connect-demo
+package triple_protocol
+
+import (
+	"errors"
+	"io"
+	"net/http"
+	"net/url"
+)
+
+// Version is the semantic version of the triple module.
+const Version = "1.7.0-dev"
+
+// These constants are used in compile-time handshakes with triple's generated
+// code.
+const (
+	IsAtLeastVersion0_0_1 = true
+	IsAtLeastVersion0_1_0 = true
+	IsAtLeastVersion1_6_0 = true
+)
+
+// StreamType describes whether the client, server, neither, or both is
+// streaming.
+type StreamType uint8
+
+const (
+	StreamTypeUnary  StreamType = 0b00
+	StreamTypeClient StreamType = 0b01
+	StreamTypeServer StreamType = 0b10
+	StreamTypeBidi              = StreamTypeClient | StreamTypeServer
+)
+
+// StreamingHandlerConn is the server's view of a bidirectional message
+// exchange. Interceptors for streaming RPCs may wrap StreamingHandlerConns.
+//
+// Like the standard library's [http.ResponseWriter], StreamingHandlerConns write
+// response headers to the network with the first call to Send. Any subsequent
+// mutations are effectively no-ops. Handlers may mutate response trailers at
+// any time before returning. When the client has finished sending data,
+// Receive returns an error wrapping [io.EOF]. Handlers should check for this
+// using the standard library's [errors.Is].
+//
+// Headers and trailers beginning with "Triple-" and "Grpc-" are reserved for
+// use by the gRPC and Triple protocols: applications may read them but
+// shouldn't write them.
+//
+// StreamingHandlerConn implementations provided by this module guarantee that
+// all returned errors can be cast to [*Error] using the standard library's
+// [errors.As].
+//
+// StreamingHandlerConn implementations do not need to be safe for concurrent use.
+type StreamingHandlerConn interface {
+	Spec() Spec
+	Peer() Peer
+
+	Receive(interface{}) error
+	RequestHeader() http.Header
+
+	Send(interface{}) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+}
+
+// StreamingClientConn is the client's view of a bidirectional message exchange.
+// Interceptors for streaming RPCs may wrap StreamingClientConns.
+//
+// StreamingClientConns write request headers to the network with the first
+// call to Send. Any subsequent mutations are effectively no-ops. When the
+// server is done sending data, the StreamingClientConn's Receive method
+// returns an error wrapping [io.EOF]. Clients should check for this using the
+// standard library's [errors.Is]. If the server encounters an error during
+// processing, subsequent calls to the StreamingClientConn's Send method will
+// return an error wrapping [io.EOF]; clients may then call Receive to unmarshal
+// the error.
+//
+// Headers and trailers beginning with "Triple-" and "Grpc-" are reserved for
+// use by the gRPC and Triple protocols: applications may read them but
+// shouldn't write them.
+//
+// StreamingClientConn implementations provided by this module guarantee that
+// all returned errors can be cast to [*Error] using the standard library's
+// [errors.As].
+//
+// In order to support bidirectional streaming RPCs, all StreamingClientConn
+// implementations must support limited concurrent use. See the comments on
+// each group of methods for details.
+type StreamingClientConn interface {
+	// Spec and Peer must be safe to call concurrently with all other methods.
+	Spec() Spec
+	Peer() Peer
+
+	// Send, RequestHeader, and CloseRequest may race with each other, but must
+	// be safe to call concurrently with all other methods.
+	Send(interface{}) error
+	RequestHeader() http.Header
+	CloseRequest() error
+
+	// Receive, ResponseHeader, ResponseTrailer, and CloseResponse may race with
+	// each other, but must be safe to call concurrently with all other methods.
+	Receive(interface{}) error
+	ResponseHeader() http.Header
+	ResponseTrailer() http.Header
+	CloseResponse() error
+}
+
+// Request is a wrapper around a generated request message. It provides
+// access to metadata like headers and the RPC specification, as well as
+// strongly-typed access to the message itself.
+type Request struct {
+	Msg interface{}
+
+	spec   Spec
+	peer   Peer
+	header http.Header
+}
+
+// NewRequest wraps a generated request message.
+func NewRequest(message interface{}) *Request {
+	return &Request{
+		Msg: message,
+		// Initialized lazily so we don't allocate unnecessarily.
+		header: nil,
+	}
+}
+
+// Any returns the concrete request message as an empty interface, so that
+// *Request implements the [AnyRequest] interface.
+func (r *Request) Any() interface{} {
+	return r.Msg
+}
+
+// Spec returns a description of this RPC.
+func (r *Request) Spec() Spec {
+	return r.spec
+}
+
+// Peer describes the other party for this RPC.
+func (r *Request) Peer() Peer {
+	return r.peer
+}
+
+// Header returns the HTTP headers for this request. Headers beginning with
+// "Triple-" and "Grpc-" are reserved for use by the Triple and gRPC
+// protocols: applications may read them but shouldn't write them.
+func (r *Request) Header() http.Header {
+	if r.header == nil {
+		r.header = make(http.Header)
+	}
+	return r.header
+}
+
+// internalOnly implements AnyRequest.
+func (r *Request) internalOnly() {}
+
+// AnyRequest is the common method set of every [Request], regardless of type
+// parameter. It's used in unary interceptors.
+//
+// Headers and trailers beginning with "Triple-" and "Grpc-" are reserved for
+// use by the gRPC and Triple protocols: applications may read them but
+// shouldn't write them.
+//
+// To preserve our ability to add methods to this interface without breaking
+// backward compatibility, only types defined in this package can implement
+// AnyRequest.
+type AnyRequest interface {
+	Any() interface{}
+	Spec() Spec
+	Peer() Peer
+	Header() http.Header
+
+	internalOnly()
+}
+
+// Response is a wrapper around a generated response message. It provides
+// access to metadata like headers and trailers, as well as strongly-typed
+// access to the message itself.
+type Response struct {
+	Msg interface{}
+
+	header  http.Header
+	trailer http.Header
+}
+
+// NewResponse wraps a generated response message.
+func NewResponse(message interface{}) *Response {
+	return &Response{
+		Msg: message,
+		// Initialized lazily so we don't allocate unnecessarily.
+		header:  nil,
+		trailer: nil,
+	}
+}
+
+// Any returns the concrete response message as an empty interface, so that
+// *Response implements the [AnyResponse] interface.
+func (r *Response) Any() interface{} {
+	return r.Msg
+}
+
+// Header returns the HTTP headers for this response. Headers beginning with
+// "Triple-" and "Grpc-" are reserved for use by the Triple and gRPC
+// protocols: applications may read them but shouldn't write them.
+func (r *Response) Header() http.Header {
+	if r.header == nil {
+		r.header = make(http.Header)
+	}
+	return r.header
+}
+
+// Trailer returns the trailers for this response. Depending on the underlying
+// RPC protocol, trailers may be sent as HTTP trailers or a protocol-specific
+// block of in-body metadata.
+//
+// Trailers beginning with "Triple-" and "Grpc-" are reserved for use by the
+// Triple and gRPC protocols: applications may read them but shouldn't write
+// them.
+func (r *Response) Trailer() http.Header {
+	if r.trailer == nil {
+		r.trailer = make(http.Header)
+	}
+	return r.trailer
+}
+
+// internalOnly implements AnyResponse.
+func (r *Response) internalOnly() {}
+
+// AnyResponse is the common method set of every [Response], regardless of type
+// parameter. It's used in unary interceptors.
+//
+// Headers and trailers beginning with "Triple-" and "Grpc-" are reserved for
+// use by the Triple and Grpc protocols: applications may read them but
+// shouldn't write them.
+//
+// To preserve our ability to add methods to this interface without breaking
+// backward compatibility, only types defined in this package can implement
+// AnyResponse.
+type AnyResponse interface {
+	Any() interface{}
+	Header() http.Header
+	Trailer() http.Header
+
+	internalOnly()
+}
+
+// HTTPClient is the interface triple expects HTTP clients to implement. The
+// standard library's *http.Client implements HTTPClient.
+type HTTPClient interface {
+	Do(*http.Request) (*http.Response, error)
+}
+
+// Spec is a description of a client call or a handler invocation.
+//
+// If you're using Protobuf, protoc-gen-triple-go generates a constant for the
+// fully-qualified Procedure corresponding to each RPC in your schema.
+type Spec struct {
+	StreamType       StreamType
+	Procedure        string // for example, "/acme.foo.v1.FooService/Bar"
+	IsClient         bool   // otherwise we're in a handler
+	IdempotencyLevel IdempotencyLevel
+}
+
+// Peer describes the other party to an RPC.
+//
+// When accessed client-side, Addr contains the host or host:port from the
+// server's URL. When accessed server-side, Addr contains the client's address
+// in IP:port format.
+//
+// On both the client and the server, Protocol is the RPC protocol in use.
+// Currently, it's either [ProtocolTriple], [ProtocolGRPC], or
+// todo: Should we support ProtocolGRPCWeb?
+// [ProtocolGRPCWeb], but additional protocols may be added in the future.
+//
+// Query contains the query parameters for the request. For the server, this
+// will reflect the actual query parameters sent. For the client, it is unset.
+type Peer struct {
+	Addr     string
+	Protocol string
+	Query    url.Values // server-only
+}
+
+func newPeerFromURL(url *url.URL, protocol string) Peer {
+	return Peer{
+		Addr:     url.Host,
+		Protocol: protocol,
+	}
+}
+
+// handlerConnCloser extends HandlerConn with a method for handlers to
+// terminate the message exchange (and optionally send an error to the client).
+type handlerConnCloser interface {
+	StreamingHandlerConn
+
+	Close(error) error
+}
+
+// receiveUnaryResponse unmarshals a message from a StreamingClientConn, then
+// envelopes the message and attaches headers and trailers. It attempts to
+// consume the response stream and isn't appropriate when receiving multiple
+// messages.
+func receiveUnaryResponse(conn StreamingClientConn, response AnyResponse) error {
+	resp, ok := response.(*Response)
+	if !ok {
+		// todo: add a more reasonable sentence
+		panic("wrong type")
+	}
+	if err := conn.Receive(resp.Msg); err != nil {
+		return err
+	}
+	// In a well-formed stream, the response message may be followed by a block
+	// of in-stream trailers or HTTP trailers. To ensure that we receive the
+	// trailers, try to read another message from the stream.
+	// if err := conn.Receive(new(T)); err == nil {
+	// todo:// maybe using copy method
+	if err := conn.Receive(resp.Msg); err == nil {
+		return NewError(CodeUnknown, errors.New("unary stream has multiple messages"))
+	} else if err != nil && !errors.Is(err, io.EOF) {
+		return NewError(CodeUnknown, err)
+	}
+	resp.header = conn.ResponseHeader()
+	resp.trailer = conn.ResponseTrailer()
+	return nil
+}
+
+// IsEnded is a convenient function indicating the end of stream. It is introduced to not expose
+// io.EOF to beginners. Please refer to https://github.com/apache/dubbo-go/pull/2416#discussion_r1318558801
+func IsEnded(err error) bool {
+	return errors.Is(err, io.EOF)
+}
diff --git a/protocol/triple/triple_protocol/triple_ext_test.go b/protocol/triple/triple_protocol/triple_ext_test.go
new file mode 100644
index 0000000..5621cf9
--- /dev/null
+++ b/protocol/triple/triple_protocol/triple_ext_test.go
@@ -0,0 +1,2349 @@
+// Copyright 2021-2023 Buf Technologies, Inc.
+//
+// Licensed under the Apache License, Version 2.0 (the "License");
+// you may not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//	http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing, software
+// distributed under the License is distributed on an "AS IS" BASIS,
+// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+// See the License for the specific language governing permissions and
+// limitations under the License.
+package triple_protocol_test
+
+import (
+	"bytes"
+	"compress/flate"
+	"compress/gzip"
+	"context"
+	"encoding/binary"
+	"errors"
+	"fmt"
+	"io"
+	"math"
+	"math/rand"
+	"net/http"
+	"net/http/httptest"
+	"strings"
+	"sync"
+	"testing"
+	"time"
+)
+
+import (
+	"google.golang.org/protobuf/proto"
+
+	"google.golang.org/protobuf/reflect/protoregistry"
+)
+
+import (
+	triple "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/assert"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/import/v1/importv1connect"
+	pingv1 "dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol/internal/gen/proto/connect/ping/v1/pingv1connect"
+)
+
+const errorMessage = "oh no"
+
+// The ping server implementation used in the tests returns errors if the
+// client doesn't set a header, and the server sets headers and trailers on the
+// response.
+const (
+	headerValue                 = "some header value"
+	trailerValue                = "some trailer value"
+	clientHeader                = "Connect-Client-Header"
+	handlerHeader               = "Connect-Handler-Header"
+	handlerTrailer              = "Connect-Handler-Trailer"
+	clientMiddlewareErrorHeader = "Connect-Trigger-HTTP-Error"
+)
+
+func TestServer(t *testing.T) {
+	t.Parallel()
+	testPing := func(t *testing.T, client pingv1connect.PingServiceClient) { //nolint:thelper
+		t.Run("ping", func(t *testing.T) {
+			num := int64(42)
+			request := triple.NewRequest(&pingv1.PingRequest{Number: num})
+			request.Header().Set(clientHeader, headerValue)
+			expect := &pingv1.PingResponse{Number: num}
+			msg := &pingv1.PingResponse{}
+			response := triple.NewResponse(msg)
+			err := client.Ping(context.Background(), request, response)
+			assert.Nil(t, err)
+			assert.Equal(t, response.Msg.(*pingv1.PingResponse), expect)
+			assert.Equal(t, response.Header().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, response.Trailer().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("zero_ping", func(t *testing.T) {
+			request := triple.NewRequest(&pingv1.PingRequest{})
+			request.Header().Set(clientHeader, headerValue)
+			msg := &pingv1.PingResponse{}
+			response := triple.NewResponse(msg)
+			err := client.Ping(context.Background(), request, response)
+			assert.Nil(t, err)
+			var expect pingv1.PingResponse
+			assert.Equal(t, msg, &expect)
+			assert.Equal(t, response.Header().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, response.Trailer().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("large_ping", func(t *testing.T) {
+			// Using a large payload splits the request and response over multiple
+			// packets, ensuring that we're managing HTTP readers and writers
+			// correctly.
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			hellos := strings.Repeat("hello", 1024*1024) // ~5mb
+			request := triple.NewRequest(&pingv1.PingRequest{Text: hellos})
+			request.Header().Set(clientHeader, headerValue)
+			msg := &pingv1.PingResponse{}
+			response := triple.NewResponse(msg)
+			err := client.Ping(context.Background(), request, response)
+			assert.Nil(t, err)
+			assert.Equal(t, msg.Text, hellos)
+			assert.Equal(t, response.Header().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, response.Trailer().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("ping_error", func(t *testing.T) {
+			err := client.Ping(
+				context.Background(),
+				triple.NewRequest(&pingv1.PingRequest{}),
+				triple.NewResponse(&pingv1.PingResponse{}),
+			)
+			assert.Equal(t, triple.CodeOf(err), triple.CodeInvalidArgument)
+		})
+		t.Run("ping_timeout", func(t *testing.T) {
+			ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(-time.Second))
+			defer cancel()
+			request := triple.NewRequest(&pingv1.PingRequest{})
+			request.Header().Set(clientHeader, headerValue)
+			err := client.Ping(ctx, request, triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeDeadlineExceeded)
+		})
+	}
+	testSum := func(t *testing.T, client pingv1connect.PingServiceClient) { //nolint:thelper
+		t.Run("sum", func(t *testing.T) {
+			const (
+				upTo   = 10
+				expect = 55 // 1+10 + 2+9 + ... + 5+6 = 55
+			)
+			stream, err := client.Sum(context.Background())
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			for i := int64(1); i <= upTo; i++ {
+				err := stream.Send(&pingv1.SumRequest{Number: i})
+				assert.Nil(t, err, assert.Sprintf("send %d", i))
+			}
+			msg := &pingv1.SumResponse{}
+			response := triple.NewResponse(msg)
+			err = stream.CloseAndReceive(response)
+			assert.Nil(t, err)
+			assert.Equal(t, msg.Sum, int64(expect))
+			assert.Equal(t, response.Header().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, response.Trailer().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("sum_error", func(t *testing.T) {
+			stream, err := client.Sum(context.Background())
+			assert.Nil(t, err)
+			if err := stream.Send(&pingv1.SumRequest{Number: 1}); err != nil {
+				assert.ErrorIs(t, err, io.EOF)
+				assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+			}
+			err = stream.CloseAndReceive(triple.NewResponse(&pingv1.SumResponse{}))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeInvalidArgument)
+		})
+		t.Run("sum_close_and_receive_without_send", func(t *testing.T) {
+			stream, err := client.Sum(context.Background())
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			msg := &pingv1.SumResponse{}
+			got := triple.NewResponse(msg)
+			err = stream.CloseAndReceive(got)
+			assert.Nil(t, err)
+			assert.Equal(t, msg, &pingv1.SumResponse{}) // receive header only stream
+			assert.Equal(t, got.Header().Values(handlerHeader), []string{headerValue})
+		})
+	}
+	testCountUp := func(t *testing.T, client pingv1connect.PingServiceClient) { //nolint:thelper
+		t.Run("count_up", func(t *testing.T) {
+			const upTo = 5
+			got := make([]int64, 0, upTo)
+			expect := make([]int64, 0, upTo)
+			for i := 1; i <= upTo; i++ {
+				expect = append(expect, int64(i))
+			}
+			request := triple.NewRequest(&pingv1.CountUpRequest{Number: upTo})
+			request.Header().Set(clientHeader, headerValue)
+			stream, err := client.CountUp(context.Background(), request)
+			assert.Nil(t, err)
+			for stream.Receive(&pingv1.CountUpResponse{}) {
+				msg := stream.Msg().(*pingv1.CountUpResponse)
+				got = append(got, msg.Number)
+			}
+			assert.Nil(t, stream.Err())
+			assert.Nil(t, stream.Close())
+			assert.Equal(t, got, expect)
+		})
+		t.Run("count_up_error", func(t *testing.T) {
+			stream, err := client.CountUp(
+				context.Background(),
+				triple.NewRequest(&pingv1.CountUpRequest{Number: 1}),
+			)
+			assert.Nil(t, err)
+			for stream.Receive(&pingv1.CountUpResponse{}) {
+				t.Fatalf("expected error, shouldn't receive any messages")
+			}
+			assert.Equal(
+				t,
+				triple.CodeOf(stream.Err()),
+				triple.CodeInvalidArgument,
+			)
+		})
+		t.Run("count_up_timeout", func(t *testing.T) {
+			ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(-time.Second))
+			defer cancel()
+			_, err := client.CountUp(ctx, triple.NewRequest(&pingv1.CountUpRequest{Number: 1}))
+			assert.NotNil(t, err)
+			assert.Equal(t, triple.CodeOf(err), triple.CodeDeadlineExceeded)
+		})
+	}
+	testCumSum := func(t *testing.T, client pingv1connect.PingServiceClient, expectSuccess bool) { //nolint:thelper
+		t.Run("cumsum", func(t *testing.T) {
+			send := []int64{3, 5, 1}
+			expect := []int64{3, 8, 9}
+			var got []int64
+			stream, err := client.CumSum(context.Background())
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			if !expectSuccess { // server doesn't support HTTP/2
+				failNoHTTP2(t, stream)
+				return
+			}
+			var wg sync.WaitGroup
+			wg.Add(2)
+			go func() {
+				defer wg.Done()
+				for i, n := range send {
+					err := stream.Send(&pingv1.CumSumRequest{Number: n})
+					assert.Nil(t, err, assert.Sprintf("send error #%d", i))
+				}
+				assert.Nil(t, stream.CloseRequest())
+			}()
+			go func() {
+				defer wg.Done()
+				for {
+					msg := &pingv1.CumSumResponse{}
+					err := stream.Receive(msg)
+					if errors.Is(err, io.EOF) {
+						break
+					}
+					assert.Nil(t, err)
+					got = append(got, msg.Sum)
+				}
+				assert.Nil(t, stream.CloseResponse())
+			}()
+			wg.Wait()
+			assert.Equal(t, got, expect)
+			assert.Equal(t, stream.ResponseHeader().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, stream.ResponseTrailer().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("cumsum_error", func(t *testing.T) {
+			stream, err := client.CumSum(context.Background())
+			assert.Nil(t, err)
+			if !expectSuccess { // server doesn't support HTTP/2
+				failNoHTTP2(t, stream)
+				return
+			}
+			if err := stream.Send(&pingv1.CumSumRequest{Number: 42}); err != nil {
+				assert.ErrorIs(t, err, io.EOF)
+				assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+			}
+			// We didn't send the headers the server expects, so we should now get an
+			// error.
+			err = stream.Receive(&pingv1.CumSumResponse{})
+			assert.Equal(t, triple.CodeOf(err), triple.CodeInvalidArgument)
+			assert.True(t, triple.IsWireError(err))
+		})
+		t.Run("cumsum_empty_stream", func(t *testing.T) {
+			stream, err := client.CumSum(context.Background())
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			if !expectSuccess { // server doesn't support HTTP/2
+				failNoHTTP2(t, stream)
+				return
+			}
+			// Deliberately closing with calling Send to test the behavior of Receive.
+			// This test case is based on the grpc interop tests.
+			assert.Nil(t, stream.CloseRequest())
+			response := &pingv1.CumSumResponse{}
+			err = stream.Receive(response)
+			assert.True(t, errors.Is(err, io.EOF))
+			assert.False(t, triple.IsWireError(err))
+			assert.Nil(t, stream.CloseResponse()) // clean-up the stream
+		})
+		t.Run("cumsum_cancel_after_first_response", func(t *testing.T) {
+			ctx, cancel := context.WithCancel(context.Background())
+			stream, err := client.CumSum(ctx)
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			if !expectSuccess { // server doesn't support HTTP/2
+				failNoHTTP2(t, stream)
+				cancel()
+				return
+			}
+			var got []int64
+			expect := []int64{42}
+			if err := stream.Send(&pingv1.CumSumRequest{Number: 42}); err != nil {
+				assert.ErrorIs(t, err, io.EOF)
+				assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+			}
+			msg := &pingv1.CumSumResponse{}
+			err = stream.Receive(msg)
+			assert.Nil(t, err)
+			got = append(got, msg.Sum)
+			cancel()
+			err = stream.Receive(&pingv1.CumSumResponse{})
+			assert.Equal(t, triple.CodeOf(err), triple.CodeCanceled)
+			assert.Equal(t, got, expect)
+			assert.False(t, triple.IsWireError(err))
+		})
+		t.Run("cumsum_cancel_before_send", func(t *testing.T) {
+			ctx, cancel := context.WithCancel(context.Background())
+			stream, err := client.CumSum(ctx)
+			assert.Nil(t, err)
+			stream.RequestHeader().Set(clientHeader, headerValue)
+			assert.Nil(t, stream.Send(&pingv1.CumSumRequest{Number: 8}))
+			cancel()
+			// On a subsequent send, ensure that we are still catching context
+			// cancellations.
+			err = stream.Send(&pingv1.CumSumRequest{Number: 19})
+			assert.Equal(t, triple.CodeOf(err), triple.CodeCanceled, assert.Sprintf("%v", err))
+			assert.False(t, triple.IsWireError(err))
+		})
+	}
+	testErrors := func(t *testing.T, client pingv1connect.PingServiceClient) { //nolint:thelper
+		assertIsHTTPMiddlewareError := func(tb testing.TB, err error) {
+			tb.Helper()
+			assert.NotNil(tb, err)
+			var tripleErr *triple.Error
+			assert.True(tb, errors.As(err, &tripleErr))
+			expect := newHTTPMiddlewareError()
+			assert.Equal(tb, tripleErr.Code(), expect.Code())
+			assert.Equal(tb, tripleErr.Message(), expect.Message())
+			for k, v := range expect.Meta() {
+				assert.Equal(tb, tripleErr.Meta().Values(k), v)
+			}
+			assert.Equal(tb, len(tripleErr.Details()), len(expect.Details()))
+		}
+		t.Run("errors", func(t *testing.T) {
+			request := triple.NewRequest(&pingv1.FailRequest{
+				Code: int32(triple.CodeResourceExhausted),
+			})
+			request.Header().Set(clientHeader, headerValue)
+			response := triple.NewResponse(&pingv1.FailResponse{})
+			err := client.Fail(context.Background(), request, response)
+			assert.NotNil(t, err)
+			var tripleErr *triple.Error
+			ok := errors.As(err, &tripleErr)
+			assert.True(t, ok, assert.Sprintf("conversion to *triple.Error"))
+			assert.True(t, triple.IsWireError(err))
+			assert.Equal(t, tripleErr.Code(), triple.CodeResourceExhausted)
+			assert.Equal(t, tripleErr.Error(), "resource_exhausted: "+errorMessage)
+			assert.Zero(t, tripleErr.Details())
+			assert.Equal(t, tripleErr.Meta().Values(handlerHeader), []string{headerValue})
+			assert.Equal(t, tripleErr.Meta().Values(handlerTrailer), []string{trailerValue})
+		})
+		t.Run("middleware_errors_unary", func(t *testing.T) {
+			request := triple.NewRequest(&pingv1.PingRequest{})
+			request.Header().Set(clientMiddlewareErrorHeader, headerValue)
+			res := triple.NewResponse(&pingv1.PingResponse{})
+			err := client.Ping(context.Background(), request, res)
+			assertIsHTTPMiddlewareError(t, err)
+		})
+		//t.Run("middleware_errors_streaming", func(t *testing.T) {
+		//	request := triple.NewRequest(&pingv1.CountUpRequest{Number: 10})
+		//	request.Header().Set(clientMiddlewareErrorHeader, headerValue)
+		//	stream, err := client.CountUp(context.Background(), request)
+		//	assert.Nil(t, err)
+		//	assert.False(t, stream.Receive(&pingv1.CountUpResponse{}))
+		//	assertIsHTTPMiddlewareError(t, stream.Err())
+		//})
+	}
+	testMatrix := func(t *testing.T, server *httptest.Server, bidi bool) { //nolint:thelper
+		run := func(t *testing.T, stream bool, opts ...triple.ClientOption) {
+			t.Helper()
+			client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, opts...)
+			testPing(t, client)
+			if !stream {
+				return
+			}
+			testSum(t, client)
+			testCountUp(t, client)
+			testCumSum(t, client, bidi)
+			testErrors(t, client)
+		}
+		t.Run("triple", func(t *testing.T) {
+			t.Run("proto", func(t *testing.T) {
+				run(t, false, triple.WithTriple())
+			})
+			t.Run("proto_gzip", func(t *testing.T) {
+				run(t, false, triple.WithTriple(), triple.WithSendGzip())
+			})
+			t.Run("json_gzip", func(t *testing.T) {
+				run(
+					t,
+					false,
+					triple.WithTriple(),
+					triple.WithProtoJSON(),
+					triple.WithSendGzip(),
+				)
+			})
+		})
+		t.Run("grpc", func(t *testing.T) {
+			t.Run("proto", func(t *testing.T) {
+				run(t, true)
+			})
+			t.Run("proto_gzip", func(t *testing.T) {
+				run(t, true, triple.WithSendGzip())
+			})
+			t.Run("json_gzip", func(t *testing.T) {
+				run(
+					t,
+					true,
+					triple.WithProtoJSON(),
+					triple.WithSendGzip(),
+				)
+			})
+		})
+	}
+
+	mux := http.NewServeMux()
+	pingRoute, pingHandler := pingv1connect.NewPingServiceHandler(
+		pingServer{checkMetadata: true},
+	)
+	errorWriter := triple.NewErrorWriter()
+	// Add some net/http middleware to the ping service so we can also exercise ErrorWriter.
+	mux.Handle(pingRoute, http.HandlerFunc(func(response http.ResponseWriter, request *http.Request) {
+		if request.Header.Get(clientMiddlewareErrorHeader) != "" {
+			defer request.Body.Close()
+			if _, err := io.Copy(io.Discard, request.Body); err != nil {
+				t.Errorf("drain request body: %v", err)
+			}
+			if !errorWriter.IsSupported(request) {
+				t.Errorf("ErrorWriter doesn't support Content-Type %q", request.Header.Get("Content-Type"))
+			}
+			if err := errorWriter.Write(response, request, newHTTPMiddlewareError()); err != nil {
+				t.Errorf("send RPC error from HTTP middleware: %v", err)
+			}
+			return
+		}
+		pingHandler.ServeHTTP(response, request)
+	}))
+
+	t.Run("http1", func(t *testing.T) {
+		t.Parallel()
+		server := httptest.NewServer(mux)
+		defer server.Close()
+		testMatrix(t, server, false /* bidi */)
+	})
+	t.Run("http2", func(t *testing.T) {
+		t.Parallel()
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		defer server.Close()
+		testMatrix(t, server, true /* bidi */)
+	})
+}
+
+func TestConcurrentStreams(t *testing.T) {
+	if testing.Short() {
+		t.Skipf("skipping %s test in short mode", t.Name())
+	}
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+	var done, start sync.WaitGroup
+	start.Add(1)
+	for i := 0; i < 100; i++ {
+		done.Add(1)
+		go func() {
+			defer done.Done()
+			client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+			var total int64
+			sum, err := client.CumSum(context.Background())
+			assert.Nil(t, err)
+			start.Wait()
+			for i := 0; i < 100; i++ {
+				num := rand.Int63n(1000) //nolint: gosec
+				total += num
+				if err := sum.Send(&pingv1.CumSumRequest{Number: num}); err != nil {
+					t.Errorf("failed to send request: %v", err)
+					break
+				}
+				resp := &pingv1.CumSumResponse{}
+				err := sum.Receive(resp)
+				if err != nil {
+					t.Errorf("failed to receive from stream: %v", err)
+					break
+				}
+				if total != resp.Sum {
+					t.Errorf("expected %d == %d", total, resp.Sum)
+					break
+				}
+			}
+			if err := sum.CloseRequest(); err != nil {
+				t.Errorf("failed to close request: %v", err)
+			}
+			if err := sum.CloseResponse(); err != nil {
+				t.Errorf("failed to close response: %v", err)
+			}
+		}()
+	}
+	start.Done()
+	done.Wait()
+}
+
+func TestHeaderBasic(t *testing.T) {
+	t.Parallel()
+	const (
+		key  = "Test-Key"
+		cval = "client value"
+		hval = "client value"
+	)
+
+	pingServer := &pluggablePingServer{
+		ping: func(ctx context.Context, request *triple.Request) (*triple.Response, error) {
+			assert.Equal(t, request.Header().Get(key), cval)
+			response := triple.NewResponse(&pingv1.PingResponse{})
+			response.Header().Set(key, hval)
+			return response, nil
+		},
+	}
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer))
+	server := httptest.NewServer(mux)
+	defer server.Close()
+
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+	request := triple.NewRequest(&pingv1.PingRequest{})
+	request.Header().Set(key, cval)
+	response := triple.NewResponse(&pingv1.PingResponse{})
+	err := client.Ping(context.Background(), request, response)
+	assert.Nil(t, err)
+	assert.Equal(t, response.Header().Get(key), hval)
+}
+
+func TestTimeoutParsing(t *testing.T) {
+	t.Parallel()
+	const timeout = 10 * time.Minute
+	pingServer := &pluggablePingServer{
+		ping: func(ctx context.Context, request *triple.Request) (*triple.Response, error) {
+			deadline, ok := ctx.Deadline()
+			assert.True(t, ok)
+			remaining := time.Until(deadline)
+			assert.True(t, remaining > 0)
+			assert.True(t, remaining <= timeout)
+			return triple.NewResponse(&pingv1.PingResponse{}), nil
+		},
+	}
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer))
+	server := httptest.NewServer(mux)
+	defer server.Close()
+
+	ctx, cancel := context.WithTimeout(context.Background(), timeout)
+	defer cancel()
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+	response := triple.NewResponse(&pingv1.PingResponse{})
+	err := client.Ping(ctx, triple.NewRequest(&pingv1.PingRequest{}), response)
+	assert.Nil(t, err)
+}
+
+func TestFailCodec(t *testing.T) {
+	t.Parallel()
+	handler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {})
+	server := httptest.NewServer(handler)
+	defer server.Close()
+	client := pingv1connect.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+		triple.WithCodec(failCodec{}),
+	)
+	stream, _ := client.CumSum(context.Background())
+	err := stream.Send(&pingv1.CumSumRequest{})
+	var tripleErr *triple.Error
+	assert.NotNil(t, err)
+	assert.True(t, errors.As(err, &tripleErr))
+	assert.Equal(t, tripleErr.Code(), triple.CodeInternal)
+}
+
+func TestContextError(t *testing.T) {
+	t.Parallel()
+	handler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {})
+	server := httptest.NewServer(handler)
+	defer server.Close()
+	client := pingv1connect.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+	)
+	ctx, cancel := context.WithCancel(context.Background())
+	cancel()
+	stream, err := client.CumSum(ctx)
+	assert.Nil(t, err)
+	err = stream.Send(nil)
+	var tripleErr *triple.Error
+	assert.NotNil(t, err)
+	assert.True(t, errors.As(err, &tripleErr))
+	assert.Equal(t, tripleErr.Code(), triple.CodeCanceled)
+	assert.False(t, triple.IsWireError(err))
+}
+
+func TestGRPCMarshalStatusError(t *testing.T) {
+	t.Parallel()
+
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{},
+		triple.WithCodec(failCodec{}),
+	))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	defer server.Close()
+
+	assertInternalError := func(tb testing.TB, opts ...triple.ClientOption) {
+		tb.Helper()
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, opts...)
+		request := triple.NewRequest(&pingv1.FailRequest{Code: int32(triple.CodeResourceExhausted)})
+		res := triple.NewResponse(&pingv1.FailResponse{})
+		err := client.Fail(context.Background(), request, res)
+		tb.Log(err)
+		assert.NotNil(t, err)
+		var tripleErr *triple.Error
+		ok := errors.As(err, &tripleErr)
+		assert.True(t, ok)
+		assert.Equal(t, tripleErr.Code(), triple.CodeInternal)
+		assert.True(
+			t,
+			strings.HasSuffix(tripleErr.Message(), ": boom"),
+		)
+	}
+
+	// Only applies to gRPC protocols, where we're marshaling the Status protobuf
+	// message to binary.
+	assertInternalError(t)
+}
+
+func TestGRPCMissingTrailersError(t *testing.T) {
+	t.Parallel()
+
+	trimTrailers := func(handler http.Handler) http.Handler {
+		return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+			r.Header.Del("Te")
+			handler.ServeHTTP(&trimTrailerWriter{w: w}, r)
+		})
+	}
+
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{checkMetadata: true},
+	))
+	server := httptest.NewUnstartedServer(trimTrailers(mux))
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+
+	assertErrorNoTrailers := func(t *testing.T, err error) {
+		t.Helper()
+		assert.NotNil(t, err)
+		var tripleErr *triple.Error
+		ok := errors.As(err, &tripleErr)
+		assert.True(t, ok)
+		assert.Equal(t, tripleErr.Code(), triple.CodeInternal)
+		assert.True(
+			t,
+			strings.HasSuffix(tripleErr.Message(), "gRPC protocol error: no Grpc-Status trailer"),
+		)
+	}
+
+	assertNilOrEOF := func(t *testing.T, err error) {
+		t.Helper()
+		if err != nil {
+			assert.ErrorIs(t, err, io.EOF)
+		}
+	}
+
+	t.Run("ping", func(t *testing.T) {
+		t.Parallel()
+		request := triple.NewRequest(&pingv1.PingRequest{Number: 1, Text: "foobar"})
+		res := triple.NewResponse(&pingv1.PingResponse{})
+		err := client.Ping(context.Background(), request, res)
+		assertErrorNoTrailers(t, err)
+	})
+	t.Run("sum", func(t *testing.T) {
+		t.Parallel()
+		stream, err := client.Sum(context.Background())
+		assert.Nil(t, err)
+		err = stream.Send(&pingv1.SumRequest{Number: 1})
+		assertNilOrEOF(t, err)
+		res := triple.NewResponse(&pingv1.SumResponse{})
+		err = stream.CloseAndReceive(res)
+		assertErrorNoTrailers(t, err)
+	})
+	t.Run("count_up", func(t *testing.T) {
+		t.Parallel()
+		stream, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{Number: 10}))
+		assert.Nil(t, err)
+		assert.False(t, stream.Receive(&pingv1.CountUpResponse{}))
+		assertErrorNoTrailers(t, stream.Err())
+	})
+	t.Run("cumsum", func(t *testing.T) {
+		t.Parallel()
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		assertNilOrEOF(t, stream.Send(&pingv1.CumSumRequest{Number: 10}))
+		err = stream.Receive(&pingv1.CumSumResponse{})
+		assertErrorNoTrailers(t, err)
+		assert.Nil(t, stream.CloseResponse())
+	})
+	t.Run("cumsum_empty_stream", func(t *testing.T) {
+		t.Parallel()
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.CloseRequest())
+		err = stream.Receive(&pingv1.CumSumResponse{})
+		assertErrorNoTrailers(t, err)
+		assert.Nil(t, stream.CloseResponse())
+	})
+}
+
+func TestUnavailableIfHostInvalid(t *testing.T) {
+	t.Parallel()
+	client := pingv1connect.NewPingServiceClient(
+		http.DefaultClient,
+		"https://api.invalid/",
+	)
+	err := client.Ping(
+		context.Background(),
+		triple.NewRequest(&pingv1.PingRequest{}),
+		triple.NewResponse(&pingv1.PingResponse{}),
+	)
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeUnavailable)
+}
+
+func TestBidiRequiresHTTP2(t *testing.T) {
+	t.Parallel()
+	handler := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		_, err := io.WriteString(w, "hello world")
+		assert.Nil(t, err)
+	})
+	server := httptest.NewServer(handler)
+	defer server.Close()
+	client := pingv1connect.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+	)
+	stream, err := client.CumSum(context.Background())
+	assert.Nil(t, err)
+	assert.Nil(t, stream.Send(&pingv1.CumSumRequest{}))
+	assert.Nil(t, stream.CloseRequest())
+	err = stream.Receive(&pingv1.CumSumResponse{})
+	assert.NotNil(t, err)
+	var tripleErr *triple.Error
+	assert.True(t, errors.As(err, &tripleErr))
+	assert.Equal(t, tripleErr.Code(), triple.CodeUnimplemented)
+	assert.True(
+		t,
+		strings.HasSuffix(tripleErr.Message(), ": bidi streams require at least HTTP/2"),
+	)
+}
+
+func TestCompressMinBytesClient(t *testing.T) {
+	t.Parallel()
+	assertContentType := func(tb testing.TB, text, expect string) {
+		tb.Helper()
+		mux := http.NewServeMux()
+		mux.Handle("/", http.HandlerFunc(func(writer http.ResponseWriter, request *http.Request) {
+			assert.Equal(tb, request.Header.Get("Content-Encoding"), expect)
+		}))
+		server := httptest.NewServer(mux)
+		tb.Cleanup(server.Close)
+		err := pingv1connect.NewPingServiceClient(
+			server.Client(),
+			server.URL,
+			triple.WithTriple(),
+			triple.WithSendGzip(),
+			triple.WithCompressMinBytes(8),
+		).Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{Text: text}), triple.NewResponse(&pingv1.PingResponse{}))
+		assert.Nil(tb, err)
+	}
+	t.Run("request_uncompressed", func(t *testing.T) {
+		t.Parallel()
+		assertContentType(t, "ping", "")
+	})
+	t.Run("request_compressed", func(t *testing.T) {
+		t.Parallel()
+		assertContentType(t, "pingping", "gzip")
+	})
+
+	t.Run("request_uncompressed", func(t *testing.T) {
+		t.Parallel()
+		assertContentType(t, "ping", "")
+	})
+	t.Run("request_compressed", func(t *testing.T) {
+		t.Parallel()
+		assertContentType(t, strings.Repeat("ping", 2), "gzip")
+	})
+}
+
+func TestCompressMinBytes(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{},
+		triple.WithCompressMinBytes(8),
+	))
+	server := httptest.NewServer(mux)
+	t.Cleanup(func() {
+		server.Close()
+	})
+	client := server.Client()
+
+	getPingResponse := func(t *testing.T, pingText string) *http.Response {
+		t.Helper()
+		request := &pingv1.PingRequest{Text: pingText}
+		requestBytes, err := proto.Marshal(request)
+		assert.Nil(t, err)
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+"/"+pingv1connect.PingServiceName+"/Ping",
+			bytes.NewReader(requestBytes),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/proto")
+		response, err := client.Do(req)
+		assert.Nil(t, err)
+		t.Cleanup(func() {
+			assert.Nil(t, response.Body.Close())
+		})
+		return response
+	}
+
+	t.Run("response_uncompressed", func(t *testing.T) {
+		t.Parallel()
+		assert.False(t, getPingResponse(t, "ping").Uncompressed) //nolint:bodyclose
+	})
+
+	t.Run("response_compressed", func(t *testing.T) {
+		t.Parallel()
+		assert.True(t, getPingResponse(t, strings.Repeat("ping", 2)).Uncompressed) //nolint:bodyclose
+	})
+}
+
+func TestCustomCompression(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	compressionName := "deflate"
+	decompressor := func() triple.Decompressor {
+		// Need to instantiate with a reader - before decompressing Reset(io.Reader) is called
+		return newDeflateReader(strings.NewReader(""))
+	}
+	compressor := func() triple.Compressor {
+		w, err := flate.NewWriter(&strings.Builder{}, flate.DefaultCompression)
+		if err != nil {
+			t.Fatalf("failed to create flate writer: %v", err)
+		}
+		return w
+	}
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{},
+		triple.WithCompression(compressionName, decompressor, compressor),
+	))
+	server := httptest.NewServer(mux)
+	defer server.Close()
+
+	client := pingv1connect.NewPingServiceClient(server.Client(),
+		server.URL,
+		triple.WithAcceptCompression(compressionName, decompressor, compressor),
+		triple.WithSendCompression(compressionName),
+	)
+	request := &pingv1.PingRequest{Text: "testing 1..2..3.."}
+	msg := &pingv1.PingResponse{}
+	response := triple.NewResponse(msg)
+	err := client.Ping(context.Background(), triple.NewRequest(request), response)
+	assert.Nil(t, err)
+	assert.Equal(t, msg, &pingv1.PingResponse{Text: request.Text})
+}
+
+func TestClientWithoutGzipSupport(t *testing.T) {
+	// See https://github.com/bufbuild/connect-go/pull/349 for why we want to
+	// support this. TL;DR is that Microsoft's dapr sidecar can't handle
+	// asymmetric compression.
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewServer(mux)
+	defer server.Close()
+
+	client := pingv1connect.NewPingServiceClient(server.Client(),
+		server.URL,
+		triple.WithAcceptCompression("gzip", nil, nil),
+		triple.WithSendGzip(),
+	)
+	request := &pingv1.PingRequest{Text: "gzip me!"}
+	err := client.Ping(context.Background(), triple.NewRequest(request), triple.NewResponse(&pingv1.PingResponse{}))
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+	assert.True(t, strings.Contains(err.Error(), "unknown compression"))
+}
+
+func TestInvalidHeaderTimeout(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewServer(mux)
+	t.Cleanup(func() {
+		server.Close()
+	})
+	getPingResponseWithTimeout := func(t *testing.T, timeout string) *http.Response {
+		t.Helper()
+		request, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+"/"+pingv1connect.PingServiceName+"/Ping",
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		request.Header.Set("Content-Type", "application/json")
+		request.Header.Set("Triple-Timeout-Ms", timeout)
+		response, err := server.Client().Do(request)
+		assert.Nil(t, err)
+		t.Cleanup(func() {
+			assert.Nil(t, response.Body.Close())
+		})
+		return response
+	}
+	t.Run("timeout_non_numeric", func(t *testing.T) {
+		t.Parallel()
+		assert.Equal(t, getPingResponseWithTimeout(t, "10s").StatusCode, http.StatusBadRequest) //nolint:bodyclose
+	})
+	t.Run("timeout_out_of_range", func(t *testing.T) {
+		t.Parallel()
+		assert.Equal(t, getPingResponseWithTimeout(t, "12345678901").StatusCode, http.StatusBadRequest) //nolint:bodyclose
+	})
+}
+
+// protocol does not know the concrete type without reflection
+
+//func TestInterceptorReturnsWrongType(t *testing.T) {
+//	t.Parallel()
+//	mux := http.NewServeMux()
+//	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+//	server := httptest.NewServer(mux)
+//	defer server.Close()
+//	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithTriple(), triple.WithInterceptors(triple.UnaryInterceptorFunc(func(next triple.UnaryFunc) triple.UnaryFunc {
+//		return func(ctx context.Context, request triple.AnyRequest, response triple.AnyResponse) error {
+//			if err := next(ctx, request, response); err != nil {
+//				return err
+//			}
+//			return nil
+//		}
+//	})))
+//	err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{Text: "hello!"}), triple.NewResponse(&pingv1.PingResponse{}))
+//	assert.NotNil(t, err)
+//	var tripleErr *triple.Error
+//	assert.True(t, errors.As(err, &tripleErr))
+//	assert.Equal(t, tripleErr.Code(), triple.CodeInternal)
+//	assert.True(t, strings.Contains(tripleErr.Message(), "unexpected client response type"))
+//}
+
+func TestHandlerWithReadMaxBytes(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	readMaxBytes := 1024
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{},
+		triple.WithReadMaxBytes(readMaxBytes),
+	))
+	readMaxBytesMatrix := func(t *testing.T, client pingv1connect.PingServiceClient, compressed bool) {
+		t.Helper()
+		t.Run("equal_read_max", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to exactly readMaxBytes (1024) - no errors expected
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1021)}
+			assert.Equal(t, proto.Size(pingRequest), readMaxBytes)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Nil(t, err)
+		})
+		t.Run("read_max_plus_one", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to readMaxBytes+1 (1025) - expect invalid argument.
+			// This will be over the limit after decompression but under with compression.
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1022)}
+			if compressed {
+				compressedSize := gzipCompressedSize(t, pingRequest)
+				assert.True(t, compressedSize < readMaxBytes, assert.Sprintf("expected compressed size %d < %d", compressedSize, readMaxBytes))
+			}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			assert.True(t, strings.HasSuffix(err.Error(), fmt.Sprintf("message size %d is larger than configured max %d", proto.Size(pingRequest), readMaxBytes)))
+		})
+		t.Run("read_max_large", func(t *testing.T) {
+			t.Parallel()
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			// Serializes to much larger than readMaxBytes (5 MiB)
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("abcde", 1024*1024)}
+			expectedSize := proto.Size(pingRequest)
+			// With gzip request compression, the error should indicate the envelope size (before decompression) is too large.
+			if compressed {
+				expectedSize = gzipCompressedSize(t, pingRequest)
+				assert.True(t, expectedSize > readMaxBytes, assert.Sprintf("expected compressed size %d > %d", expectedSize, readMaxBytes))
+			}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: message size %d is larger than configured max %d", expectedSize, readMaxBytes))
+		})
+	}
+	newHTTP2Server := func(t *testing.T) *httptest.Server {
+		t.Helper()
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		t.Cleanup(server.Close)
+		return server
+	}
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		readMaxBytesMatrix(t, client, false)
+	})
+	t.Run("connect_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendGzip())
+		readMaxBytesMatrix(t, client, true)
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		readMaxBytesMatrix(t, client, false)
+	})
+	t.Run("grpc_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendGzip())
+		readMaxBytesMatrix(t, client, true)
+	})
+}
+
+func TestHandlerWithHTTPMaxBytes(t *testing.T) {
+	// This is similar to Connect's own ReadMaxBytes option, but applied to the
+	// whole stream using the stdlib's http.MaxBytesHandler.
+	t.Parallel()
+	const readMaxBytes = 128
+	mux := http.NewServeMux()
+	pingRoute, pingHandler := pingv1connect.NewPingServiceHandler(pingServer{})
+	mux.Handle(pingRoute, http.MaxBytesHandler(pingHandler, readMaxBytes))
+	run := func(t *testing.T, client pingv1connect.PingServiceClient, compressed bool) {
+		t.Helper()
+		t.Run("below_read_max", func(t *testing.T) {
+			t.Parallel()
+			err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Nil(t, err)
+		})
+		t.Run("just_above_max", func(t *testing.T) {
+			t.Parallel()
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", readMaxBytes*10)}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			if compressed {
+				compressedSize := gzipCompressedSize(t, pingRequest)
+				assert.True(t, compressedSize < readMaxBytes, assert.Sprintf("expected compressed size %d < %d", compressedSize, readMaxBytes))
+				assert.Nil(t, err)
+				return
+			}
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+		})
+		t.Run("read_max_large", func(t *testing.T) {
+			t.Parallel()
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("abcde", 1024*1024)}
+			if compressed {
+				expectedSize := gzipCompressedSize(t, pingRequest)
+				assert.True(t, expectedSize > readMaxBytes, assert.Sprintf("expected compressed size %d > %d", expectedSize, readMaxBytes))
+			}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+		})
+	}
+	newHTTP2Server := func(t *testing.T) *httptest.Server {
+		t.Helper()
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		t.Cleanup(server.Close)
+		return server
+	}
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		run(t, client, false)
+	})
+	t.Run("connect_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendGzip())
+		run(t, client, true)
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		run(t, client, false)
+	})
+	t.Run("grpc_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendGzip())
+		run(t, client, true)
+	})
+}
+
+func TestClientWithReadMaxBytes(t *testing.T) {
+	t.Parallel()
+	createServer := func(tb testing.TB, enableCompression bool) *httptest.Server {
+		tb.Helper()
+		mux := http.NewServeMux()
+		var compressionOption triple.HandlerOption
+		if enableCompression {
+			compressionOption = triple.WithCompressMinBytes(1)
+		} else {
+			compressionOption = triple.WithCompressMinBytes(math.MaxInt)
+		}
+		mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}, compressionOption))
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		tb.Cleanup(server.Close)
+		return server
+	}
+	serverUncompressed := createServer(t, false)
+	serverCompressed := createServer(t, true)
+	readMaxBytes := 1024
+	readMaxBytesMatrix := func(t *testing.T, client pingv1connect.PingServiceClient, compressed bool) {
+		t.Helper()
+		t.Run("equal_read_max", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to exactly readMaxBytes (1024) - no errors expected
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1021)}
+			assert.Equal(t, proto.Size(pingRequest), readMaxBytes)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Nil(t, err)
+		})
+		t.Run("read_max_plus_one", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to readMaxBytes+1 (1025) - expect resource exhausted.
+			// This will be over the limit after decompression but under with compression.
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1022)}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			assert.True(t, strings.HasSuffix(err.Error(), fmt.Sprintf("message size %d is larger than configured max %d", proto.Size(pingRequest), readMaxBytes)))
+		})
+		t.Run("read_max_large", func(t *testing.T) {
+			t.Parallel()
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			// Serializes to much larger than readMaxBytes (5 MiB)
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("abcde", 1024*1024)}
+			expectedSize := proto.Size(pingRequest)
+			// With gzip response compression, the error should indicate the envelope size (before decompression) is too large.
+			if compressed {
+				expectedSize = gzipCompressedSize(t, pingRequest)
+				assert.True(t, expectedSize > readMaxBytes, assert.Sprintf("expected compressed size %d > %d", expectedSize, readMaxBytes))
+			}
+			assert.True(t, expectedSize > readMaxBytes, assert.Sprintf("expected compressed size %d > %d", expectedSize, readMaxBytes))
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: message size %d is larger than configured max %d", expectedSize, readMaxBytes))
+		})
+	}
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		client := pingv1connect.NewPingServiceClient(serverUncompressed.Client(), serverUncompressed.URL, triple.WithReadMaxBytes(readMaxBytes))
+		readMaxBytesMatrix(t, client, false)
+	})
+	t.Run("connect_gzip", func(t *testing.T) {
+		t.Parallel()
+		client := pingv1connect.NewPingServiceClient(serverCompressed.Client(), serverCompressed.URL, triple.WithReadMaxBytes(readMaxBytes))
+		readMaxBytesMatrix(t, client, true)
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		client := pingv1connect.NewPingServiceClient(serverUncompressed.Client(), serverUncompressed.URL, triple.WithReadMaxBytes(readMaxBytes))
+		readMaxBytesMatrix(t, client, false)
+	})
+	t.Run("grpc_gzip", func(t *testing.T) {
+		t.Parallel()
+		client := pingv1connect.NewPingServiceClient(serverCompressed.Client(), serverCompressed.URL, triple.WithReadMaxBytes(readMaxBytes))
+		readMaxBytesMatrix(t, client, true)
+	})
+}
+
+func TestHandlerWithSendMaxBytes(t *testing.T) {
+	t.Parallel()
+	sendMaxBytes := 1024
+	sendMaxBytesMatrix := func(t *testing.T, client pingv1connect.PingServiceClient, compressed bool) {
+		t.Helper()
+		t.Run("equal_send_max", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to exactly sendMaxBytes (1024) - no errors expected
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1021)}
+			assert.Equal(t, proto.Size(pingRequest), sendMaxBytes)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Nil(t, err)
+		})
+		t.Run("send_max_plus_one", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to sendMaxBytes+1 (1025) - expect invalid argument.
+			// This will be over the limit after decompression but under with compression.
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1022)}
+			if compressed {
+				compressedSize := gzipCompressedSize(t, pingRequest)
+				assert.True(t, compressedSize < sendMaxBytes, assert.Sprintf("expected compressed size %d < %d", compressedSize, sendMaxBytes))
+			}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			if compressed {
+				assert.Nil(t, err)
+			} else {
+				assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+				assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+				assert.True(t, strings.HasSuffix(err.Error(), fmt.Sprintf("message size %d exceeds sendMaxBytes %d", proto.Size(pingRequest), sendMaxBytes)))
+			}
+		})
+		t.Run("send_max_large", func(t *testing.T) {
+			t.Parallel()
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			// Serializes to much larger than sendMaxBytes (5 MiB)
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("abcde", 1024*1024)}
+			expectedSize := proto.Size(pingRequest)
+			// With gzip request compression, the error should indicate the envelope size (before decompression) is too large.
+			if compressed {
+				expectedSize = gzipCompressedSize(t, pingRequest)
+				assert.True(t, expectedSize > sendMaxBytes, assert.Sprintf("expected compressed size %d > %d", expectedSize, sendMaxBytes))
+			}
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			if compressed {
+				assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: compressed message size %d exceeds sendMaxBytes %d", expectedSize, sendMaxBytes))
+			} else {
+				assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: message size %d exceeds sendMaxBytes %d", expectedSize, sendMaxBytes))
+			}
+		})
+	}
+	newHTTP2Server := func(t *testing.T, compressed bool, sendMaxBytes int) *httptest.Server {
+		t.Helper()
+		mux := http.NewServeMux()
+		options := []triple.HandlerOption{triple.WithSendMaxBytes(sendMaxBytes)}
+		if compressed {
+			options = append(options, triple.WithCompressMinBytes(1))
+		} else {
+			options = append(options, triple.WithCompressMinBytes(math.MaxInt))
+		}
+		mux.Handle(pingv1connect.NewPingServiceHandler(
+			pingServer{},
+			options...,
+		))
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		t.Cleanup(server.Close)
+		return server
+	}
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t, false, sendMaxBytes)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		sendMaxBytesMatrix(t, client, false)
+	})
+	t.Run("connect_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t, true, sendMaxBytes)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		sendMaxBytesMatrix(t, client, true)
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t, false, sendMaxBytes)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		sendMaxBytesMatrix(t, client, false)
+	})
+	t.Run("grpc_gzip", func(t *testing.T) {
+		t.Parallel()
+		server := newHTTP2Server(t, true, sendMaxBytes)
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		sendMaxBytesMatrix(t, client, true)
+	})
+}
+
+func TestClientWithSendMaxBytes(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+	sendMaxBytesMatrix := func(t *testing.T, client pingv1connect.PingServiceClient, sendMaxBytes int, compressed bool) {
+		t.Helper()
+		t.Run("equal_send_max", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to exactly sendMaxBytes (1024) - no errors expected
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1021)}
+			assert.Equal(t, proto.Size(pingRequest), sendMaxBytes)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.Nil(t, err)
+		})
+		t.Run("send_max_plus_one", func(t *testing.T) {
+			t.Parallel()
+			// Serializes to sendMaxBytes+1 (1025) - expect resource exhausted.
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("a", 1022)}
+			assert.Equal(t, proto.Size(pingRequest), sendMaxBytes+1)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			if compressed {
+				assert.True(t, gzipCompressedSize(t, pingRequest) < sendMaxBytes)
+				assert.Nil(t, err, assert.Sprintf("expected nil error for compressed message < sendMaxBytes"))
+			} else {
+				assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+				assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+				assert.True(t, strings.HasSuffix(err.Error(), fmt.Sprintf("message size %d exceeds sendMaxBytes %d", proto.Size(pingRequest), sendMaxBytes)))
+			}
+		})
+		t.Run("send_max_large", func(t *testing.T) {
+			t.Parallel()
+			if testing.Short() {
+				t.Skipf("skipping %s test in short mode", t.Name())
+			}
+			// Serializes to much larger than sendMaxBytes (5 MiB)
+			pingRequest := &pingv1.PingRequest{Text: strings.Repeat("abcde", 1024*1024)}
+			expectedSize := proto.Size(pingRequest)
+			// With gzip response compression, the error should indicate the envelope size (before decompression) is too large.
+			if compressed {
+				expectedSize = gzipCompressedSize(t, pingRequest)
+			}
+			assert.True(t, expectedSize > sendMaxBytes)
+			err := client.Ping(context.Background(), triple.NewRequest(pingRequest), triple.NewResponse(&pingv1.PingResponse{}))
+			assert.NotNil(t, err, assert.Sprintf("expected non-nil error for large message"))
+			assert.Equal(t, triple.CodeOf(err), triple.CodeResourceExhausted)
+			if compressed {
+				assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: compressed message size %d exceeds sendMaxBytes %d", expectedSize, sendMaxBytes))
+			} else {
+				assert.Equal(t, err.Error(), fmt.Sprintf("resource_exhausted: message size %d exceeds sendMaxBytes %d", expectedSize, sendMaxBytes))
+			}
+		})
+	}
+	t.Run("triple", func(t *testing.T) {
+		t.Parallel()
+		sendMaxBytes := 1024
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendMaxBytes(sendMaxBytes))
+		sendMaxBytesMatrix(t, client, sendMaxBytes, false)
+	})
+	t.Run("connect_gzip", func(t *testing.T) {
+		t.Parallel()
+		sendMaxBytes := 1024
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendMaxBytes(sendMaxBytes), triple.WithSendGzip())
+		sendMaxBytesMatrix(t, client, sendMaxBytes, true)
+	})
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		sendMaxBytes := 1024
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendMaxBytes(sendMaxBytes))
+		sendMaxBytesMatrix(t, client, sendMaxBytes, false)
+	})
+	t.Run("grpc_gzip", func(t *testing.T) {
+		t.Parallel()
+		sendMaxBytes := 1024
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, triple.WithSendMaxBytes(sendMaxBytes), triple.WithSendGzip())
+		sendMaxBytesMatrix(t, client, sendMaxBytes, true)
+	})
+}
+
+func TestBidiStreamServerSendsFirstMessage(t *testing.T) {
+	t.Parallel()
+	run := func(t *testing.T, opts ...triple.ClientOption) {
+		t.Helper()
+		headersSent := make(chan struct{})
+		pingServer := &pluggablePingServer{
+			cumSum: func(ctx context.Context, stream *triple.BidiStream) error {
+				close(headersSent)
+				return nil
+			},
+		}
+		mux := http.NewServeMux()
+		mux.Handle(pingv1connect.NewPingServiceHandler(pingServer))
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		t.Cleanup(server.Close)
+
+		client := pingv1connect.NewPingServiceClient(
+			server.Client(),
+			server.URL,
+			triple.WithClientOptions(opts...),
+			triple.WithInterceptors(&assertPeerInterceptor{t}),
+		)
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		t.Cleanup(func() {
+			assert.Nil(t, stream.CloseRequest())
+			assert.Nil(t, stream.CloseResponse())
+		})
+		assert.Nil(t, stream.Send(nil))
+		select {
+		case <-time.After(time.Second):
+			t.Error("timed out to get request headers")
+		case <-headersSent:
+		}
+	}
+	t.Run("grpc", func(t *testing.T) {
+		t.Parallel()
+		run(t)
+	})
+}
+
+func TestStreamForServer(t *testing.T) {
+	t.Parallel()
+	newPingServer := func(pingServer pingv1connect.PingServiceHandler) (pingv1connect.PingServiceClient, *httptest.Server) {
+		mux := http.NewServeMux()
+		mux.Handle(pingv1connect.NewPingServiceHandler(pingServer))
+		server := httptest.NewUnstartedServer(mux)
+		server.EnableHTTP2 = true
+		server.StartTLS()
+		client := pingv1connect.NewPingServiceClient(
+			server.Client(),
+			server.URL,
+		)
+		return client, server
+	}
+	t.Run("not-proto-message", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			cumSum: func(ctx context.Context, stream *triple.BidiStream) error {
+				return stream.Conn().Send("foobar")
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(nil))
+		err = stream.Receive(&pingv1.CumSumResponse{})
+		assert.NotNil(t, err)
+		assert.Equal(t, triple.CodeOf(err), triple.CodeInternal)
+		assert.Nil(t, stream.CloseRequest())
+	})
+	t.Run("nil-message", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			cumSum: func(ctx context.Context, stream *triple.BidiStream) error {
+				return stream.Send(nil)
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(nil))
+		err = stream.Receive(&pingv1.CumSumResponse{})
+		assert.NotNil(t, err)
+		assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+		assert.Nil(t, stream.CloseRequest())
+	})
+	t.Run("get-spec", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			cumSum: func(ctx context.Context, stream *triple.BidiStream) error {
+				assert.Equal(t, stream.Spec().StreamType, triple.StreamTypeBidi)
+				assert.Equal(t, stream.Spec().Procedure, pingv1connect.PingServiceCumSumProcedure)
+				assert.False(t, stream.Spec().IsClient)
+				return nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CumSum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(nil))
+		assert.Nil(t, stream.CloseRequest())
+	})
+	t.Run("server-stream", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			countUp: func(ctx context.Context, req *triple.Request, stream *triple.ServerStream) error {
+				assert.Equal(t, stream.Conn().Spec().StreamType, triple.StreamTypeServer)
+				assert.Equal(t, stream.Conn().Spec().Procedure, pingv1connect.PingServiceCountUpProcedure)
+				assert.False(t, stream.Conn().Spec().IsClient)
+				assert.Nil(t, stream.Send(&pingv1.CountUpResponse{Number: 1}))
+				return nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{}))
+		assert.Nil(t, err)
+		assert.NotNil(t, stream)
+		assert.Nil(t, stream.Close())
+	})
+	t.Run("server-stream-send", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			countUp: func(ctx context.Context, req *triple.Request, stream *triple.ServerStream) error {
+				assert.Nil(t, stream.Send(&pingv1.CountUpResponse{Number: 1}))
+				return nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{}))
+		assert.Nil(t, err)
+		assert.True(t, stream.Receive(&pingv1.CountUpResponse{}))
+		msg := stream.Msg().(*pingv1.CountUpResponse)
+		assert.NotNil(t, msg)
+		assert.Equal(t, msg.Number, int64(1))
+		assert.Nil(t, stream.Close())
+	})
+	t.Run("server-stream-send-nil", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			countUp: func(ctx context.Context, req *triple.Request, stream *triple.ServerStream) error {
+				stream.ResponseHeader().Set("foo", "bar")
+				stream.ResponseTrailer().Set("bas", "blah")
+				assert.Nil(t, stream.Send(nil))
+				return nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.CountUp(context.Background(), triple.NewRequest(&pingv1.CountUpRequest{}))
+		assert.Nil(t, err)
+		assert.False(t, stream.Receive(&pingv1.CountUpResponse{}))
+		headers := stream.ResponseHeader()
+		assert.NotNil(t, headers)
+		assert.Equal(t, headers.Get("foo"), "bar")
+		trailers := stream.ResponseTrailer()
+		assert.NotNil(t, trailers)
+		assert.Equal(t, trailers.Get("bas"), "blah")
+		assert.Nil(t, stream.Close())
+	})
+	t.Run("client-stream", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			sum: func(ctx context.Context, stream *triple.ClientStream) (*triple.Response, error) {
+				assert.Equal(t, stream.Spec().StreamType, triple.StreamTypeClient)
+				assert.Equal(t, stream.Spec().Procedure, pingv1connect.PingServiceSumProcedure)
+				assert.False(t, stream.Spec().IsClient)
+				assert.True(t, stream.Receive(&pingv1.SumRequest{}))
+				msg := stream.Msg().(*pingv1.SumRequest)
+				assert.NotNil(t, msg)
+				assert.Equal(t, msg.Number, int64(1))
+				return triple.NewResponse(&pingv1.SumResponse{Sum: 1}), nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.Sum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(&pingv1.SumRequest{Number: 1}))
+		msg := &pingv1.SumResponse{}
+		res := triple.NewResponse(msg)
+		err = stream.CloseAndReceive(res)
+		assert.Nil(t, err)
+		assert.NotNil(t, res)
+		assert.Equal(t, msg.Sum, int64(1))
+	})
+	t.Run("client-stream-conn", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			sum: func(ctx context.Context, stream *triple.ClientStream) (*triple.Response, error) {
+				assert.NotNil(t, stream.Conn().Send("not-proto"))
+				return triple.NewResponse(&pingv1.SumResponse{}), nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.Sum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(&pingv1.SumRequest{Number: 1}))
+		res := triple.NewResponse(&pingv1.SumResponse{})
+		err = stream.CloseAndReceive(res)
+		assert.Nil(t, err)
+	})
+	t.Run("client-stream-send-msg", func(t *testing.T) {
+		t.Parallel()
+		client, server := newPingServer(&pluggablePingServer{
+			sum: func(ctx context.Context, stream *triple.ClientStream) (*triple.Response, error) {
+				assert.Nil(t, stream.Conn().Send(&pingv1.SumResponse{Sum: 2}))
+				return triple.NewResponse(&pingv1.SumResponse{}), nil
+			},
+		})
+		t.Cleanup(server.Close)
+		stream, err := client.Sum(context.Background())
+		assert.Nil(t, err)
+		assert.Nil(t, stream.Send(&pingv1.SumRequest{Number: 1}))
+		res := triple.NewResponse(&pingv1.SumResponse{})
+		err = stream.CloseAndReceive(res)
+		assert.NotNil(t, err)
+		assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+	})
+}
+
+func TestConnectHTTPErrorCodes(t *testing.T) {
+	t.Parallel()
+	checkHTTPStatus := func(t *testing.T, connectCode triple.Code, wantHttpStatus int) {
+		t.Helper()
+		mux := http.NewServeMux()
+		pluggableServer := &pluggablePingServer{
+			ping: func(_ context.Context, _ *triple.Request) (*triple.Response, error) {
+				return nil, triple.NewError(connectCode, errors.New("error"))
+			},
+		}
+		mux.Handle(pingv1connect.NewPingServiceHandler(pluggableServer))
+		server := httptest.NewServer(mux)
+		t.Cleanup(server.Close)
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+"/"+pingv1connect.PingServiceName+"/Ping",
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/json")
+		resp, err := server.Client().Do(req)
+		assert.Nil(t, err)
+		defer resp.Body.Close()
+		assert.Equal(t, wantHttpStatus, resp.StatusCode)
+		connectClient := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+		err = connectClient.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+		assert.NotNil(t, err)
+	}
+	t.Run("CodeCanceled-408", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeCanceled, 408)
+	})
+	t.Run("CodeUnknown-500", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeUnknown, 500)
+	})
+	t.Run("CodeInvalidArgument-400", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeInvalidArgument, 400)
+	})
+	t.Run("CodeDeadlineExceeded-408", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeDeadlineExceeded, 408)
+	})
+	t.Run("CodeNotFound-404", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeNotFound, 404)
+	})
+	t.Run("CodeAlreadyExists-409", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeAlreadyExists, 409)
+	})
+	t.Run("CodePermissionDenied-403", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodePermissionDenied, 403)
+	})
+	t.Run("CodeResourceExhausted-429", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeResourceExhausted, 429)
+	})
+	t.Run("CodeFailedPrecondition-412", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeFailedPrecondition, 412)
+	})
+	t.Run("CodeAborted-409", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeAborted, 409)
+	})
+	t.Run("CodeOutOfRange-400", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeOutOfRange, 400)
+	})
+	t.Run("CodeUnimplemented-404", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeUnimplemented, 404)
+	})
+	t.Run("CodeInternal-500", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeInternal, 500)
+	})
+	t.Run("CodeUnavailable-503", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeUnavailable, 503)
+	})
+	t.Run("CodeDataLoss-500", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeDataLoss, 500)
+	})
+	t.Run("CodeUnauthenticated-401", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, triple.CodeUnauthenticated, 401)
+	})
+	t.Run("100-500", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, 100, 500)
+	})
+	t.Run("0-500", func(t *testing.T) {
+		t.Parallel()
+		checkHTTPStatus(t, 0, 500)
+	})
+}
+
+func TestFailCompression(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	compressorName := "fail"
+	compressor := func() triple.Compressor { return failCompressor{} }
+	decompressor := func() triple.Decompressor { return failDecompressor{} }
+	mux.Handle(
+		pingv1connect.NewPingServiceHandler(
+			pingServer{},
+			triple.WithCompression(compressorName, decompressor, compressor),
+		),
+	)
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+	pingclient := pingv1connect.NewPingServiceClient(
+		server.Client(),
+		server.URL,
+		triple.WithAcceptCompression(compressorName, decompressor, compressor),
+		triple.WithSendCompression(compressorName),
+	)
+	err := pingclient.Ping(
+		context.Background(),
+		triple.NewRequest(&pingv1.PingRequest{
+			Text: "ping",
+		}),
+		triple.NewResponse(&pingv1.PingResponse{}),
+	)
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeInternal)
+}
+
+func TestUnflushableResponseWriter(t *testing.T) {
+	t.Parallel()
+	assertIsFlusherErr := func(t *testing.T, err error) {
+		t.Helper()
+		assert.NotNil(t, err)
+		assert.Equal(t, triple.CodeOf(err), triple.CodeInternal, assert.Sprintf("got %v", err))
+		assert.True(
+			t,
+			strings.HasSuffix(err.Error(), "unflushableWriter does not implement http.Flusher"),
+			assert.Sprintf("error doesn't reference http.Flusher: %s", err.Error()),
+		)
+	}
+	mux := http.NewServeMux()
+	path, handler := pingv1connect.NewPingServiceHandler(pingServer{})
+	wrapped := http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
+		handler.ServeHTTP(&unflushableWriter{w}, r)
+	})
+	mux.Handle(path, wrapped)
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+
+	tests := []struct {
+		name    string
+		options []triple.ClientOption
+	}{
+		{"grpc", nil},
+	}
+	for _, tt := range tests {
+		tt := tt
+		t.Run(tt.name, func(t *testing.T) {
+			t.Parallel()
+			pingclient := pingv1connect.NewPingServiceClient(server.Client(), server.URL, tt.options...)
+			stream, err := pingclient.CountUp(
+				context.Background(),
+				triple.NewRequest(&pingv1.CountUpRequest{Number: 5}),
+			)
+			if err != nil {
+				assertIsFlusherErr(t, err)
+				return
+			}
+			assert.False(t, stream.Receive(&pingv1.CountUpResponse{}))
+			assertIsFlusherErr(t, stream.Err())
+		})
+	}
+}
+
+func TestGRPCErrorMetadataIsTrailersOnly(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+
+	protoBytes, err := proto.Marshal(&pingv1.FailRequest{Code: int32(triple.CodeInternal)})
+	assert.Nil(t, err)
+	// Manually construct a gRPC prefix. Data is uncompressed, so the first byte
+	// is 0. Set the last 4 bytes to the message length.
+	var prefix [5]byte
+	binary.BigEndian.PutUint32(prefix[1:5], uint32(len(protoBytes)))
+	body := append(prefix[:], protoBytes...)
+	// Manually send off a gRPC request.
+	req, err := http.NewRequestWithContext(
+		context.Background(),
+		http.MethodPost,
+		server.URL+pingv1connect.PingServiceFailProcedure,
+		bytes.NewReader(body),
+	)
+	assert.Nil(t, err)
+	req.Header.Set("Content-Type", "application/grpc")
+	res, err := server.Client().Do(req)
+	assert.Nil(t, err)
+	assert.Equal(t, res.StatusCode, http.StatusOK)
+	assert.Equal(t, res.Header.Get("Content-Type"), "application/grpc")
+	// pingServer.Fail adds handlerHeader and handlerTrailer to the error
+	// metadata. The gRPC protocol should send all error metadata as trailers.
+	assert.Zero(t, res.Header.Get(handlerHeader))
+	assert.Zero(t, res.Header.Get(handlerTrailer))
+	_, err = io.Copy(io.Discard, res.Body)
+	assert.Nil(t, err)
+	assert.Nil(t, res.Body.Close())
+	assert.NotZero(t, res.Trailer.Get(handlerHeader))
+	assert.NotZero(t, res.Trailer.Get(handlerTrailer))
+}
+
+func TestConnectProtocolHeaderSentByDefault(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}, triple.WithRequireConnectProtocolHeader()))
+	server := httptest.NewUnstartedServer(mux)
+	server.EnableHTTP2 = true
+	server.StartTLS()
+	t.Cleanup(server.Close)
+
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+	err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+	assert.Nil(t, err)
+
+	stream, err := client.CumSum(context.Background())
+	assert.Nil(t, err)
+	assert.Nil(t, stream.Send(&pingv1.CumSumRequest{}))
+	err = stream.Receive(&pingv1.CumSumResponse{})
+	assert.Nil(t, err)
+	assert.Nil(t, stream.CloseRequest())
+	assert.Nil(t, stream.CloseResponse())
+}
+
+func TestConnectProtocolHeaderRequired(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(
+		pingServer{},
+		triple.WithRequireConnectProtocolHeader(),
+	))
+	server := httptest.NewServer(mux)
+	t.Cleanup(server.Close)
+
+	tests := []struct {
+		headers http.Header
+	}{
+		{http.Header{}},
+		{http.Header{"Connect-Protocol-Version": []string{"0"}}},
+	}
+	for _, tcase := range tests {
+		req, err := http.NewRequestWithContext(
+			context.Background(),
+			http.MethodPost,
+			server.URL+"/"+pingv1connect.PingServiceName+"/Ping",
+			strings.NewReader("{}"),
+		)
+		assert.Nil(t, err)
+		req.Header.Set("Content-Type", "application/json")
+		for k, v := range tcase.headers {
+			req.Header[k] = v
+		}
+		response, err := server.Client().Do(req)
+		assert.Nil(t, err)
+		assert.Nil(t, response.Body.Close())
+		assert.Equal(t, response.StatusCode, http.StatusBadRequest)
+	}
+}
+
+func TestAllowCustomUserAgent(t *testing.T) {
+	t.Parallel()
+
+	const customAgent = "custom"
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(&pluggablePingServer{
+		ping: func(_ context.Context, req *triple.Request) (*triple.Response, error) {
+			agent := req.Header().Get("User-Agent")
+			assert.Equal(t, agent, customAgent)
+			msg := req.Msg.(*pingv1.PingRequest)
+			return triple.NewResponse(&pingv1.PingResponse{Number: msg.Number}), nil
+		},
+	}))
+	server := httptest.NewServer(mux)
+	t.Cleanup(server.Close)
+
+	// If the user has set a User-Agent, we shouldn't clobber it.
+	tests := []struct {
+		protocol string
+		opts     []triple.ClientOption
+	}{
+		{"triple", nil},
+		{"grpc", nil},
+	}
+	for _, testCase := range tests {
+		client := pingv1connect.NewPingServiceClient(server.Client(), server.URL, testCase.opts...)
+		req := triple.NewRequest(&pingv1.PingRequest{Number: 42})
+		req.Header().Set("User-Agent", customAgent)
+		err := client.Ping(context.Background(), req, triple.NewResponse(&pingv1.PingResponse{}))
+		assert.Nil(t, err)
+	}
+}
+
+func TestBidiOverHTTP1(t *testing.T) {
+	t.Parallel()
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(pingServer{}))
+	server := httptest.NewServer(mux)
+	t.Cleanup(server.Close)
+
+	// Clients expecting a full-duplex connection that end up with a simplex
+	// HTTP/1.1 connection shouldn't hang. Instead, the server should close the
+	// TCP connection.
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+	stream, err := client.CumSum(context.Background())
+	assert.Nil(t, err)
+	if err := stream.Send(&pingv1.CumSumRequest{Number: 2}); err != nil {
+		assert.ErrorIs(t, err, io.EOF)
+	}
+	err = stream.Receive(&pingv1.CumSumResponse{})
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeUnknown)
+	assert.Equal(t, err.Error(), "unknown: HTTP status 505 HTTP Version Not Supported")
+	assert.Nil(t, stream.CloseRequest())
+	assert.Nil(t, stream.CloseResponse())
+}
+
+func TestHandlerReturnsNilResponse(t *testing.T) {
+	// When user-written handlers return nil responses _and_ nil errors, ensure
+	// that the resulting panic includes at least the name of the procedure.
+	t.Parallel()
+
+	var panics int
+	recoverPanic := func(_ context.Context, spec triple.Spec, _ http.Header, p interface{}) error {
+		panics++
+		assert.NotNil(t, p)
+		str := fmt.Sprint(p)
+		assert.True(
+			t,
+			strings.Contains(str, spec.Procedure),
+			assert.Sprintf("%q does not contain procedure %q", str, spec.Procedure),
+		)
+		return triple.NewError(triple.CodeInternal, errors.New(str))
+	}
+
+	mux := http.NewServeMux()
+	mux.Handle(pingv1connect.NewPingServiceHandler(&pluggablePingServer{
+		ping: func(ctx context.Context, req *triple.Request) (*triple.Response, error) {
+			return nil, nil //nolint: nilnil
+		},
+		sum: func(ctx context.Context, req *triple.ClientStream) (*triple.Response, error) {
+			return nil, nil //nolint: nilnil
+		},
+	}, triple.WithRecover(recoverPanic)))
+	server := httptest.NewServer(mux)
+	t.Cleanup(server.Close)
+	client := pingv1connect.NewPingServiceClient(server.Client(), server.URL)
+
+	err := client.Ping(context.Background(), triple.NewRequest(&pingv1.PingRequest{}), triple.NewResponse(&pingv1.PingResponse{}))
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeInternal)
+
+	stream, err := client.Sum(context.Background())
+	assert.Nil(t, err)
+	err = stream.CloseAndReceive(triple.NewResponse(&pingv1.SumResponse{}))
+	assert.NotNil(t, err)
+	assert.Equal(t, triple.CodeOf(err), triple.CodeInternal)
+
+	assert.Equal(t, panics, 2)
+}
+
+// TestBlankImportCodeGeneration tests that services.triple.go is generated with
+// blank import statements to services.pb.go so that the service's Descriptor is
+// available in the global proto registry.
+func TestBlankImportCodeGeneration(t *testing.T) {
+	t.Parallel()
+	desc, err := protoregistry.GlobalFiles.FindDescriptorByName(importv1connect.ImportServiceName)
+	assert.Nil(t, err)
+	assert.NotNil(t, desc)
+}
+
+type unflushableWriter struct {
+	w http.ResponseWriter
+}
+
+func (w *unflushableWriter) Header() http.Header         { return w.w.Header() }
+func (w *unflushableWriter) Write(b []byte) (int, error) { return w.w.Write(b) }
+func (w *unflushableWriter) WriteHeader(code int)        { w.w.WriteHeader(code) }
+
+func gzipCompressedSize(tb testing.TB, message proto.Message) int {
+	tb.Helper()
+	uncompressed, err := proto.Marshal(message)
+	assert.Nil(tb, err)
+	var buf bytes.Buffer
+	gzipWriter := gzip.NewWriter(&buf)
+	_, err = gzipWriter.Write(uncompressed)
+	assert.Nil(tb, err)
+	assert.Nil(tb, gzipWriter.Close())
+	return buf.Len()
+}
+
+type failCodec struct{}
+
+func (c failCodec) Name() string {
+	return "proto"
+}
+
+func (c failCodec) Marshal(message interface{}) ([]byte, error) {
+	return nil, errors.New("boom")
+}
+
+func (c failCodec) Unmarshal(data []byte, message interface{}) error {
+	protoMessage, ok := message.(proto.Message)
+	if !ok {
+		return fmt.Errorf("not protobuf: %T", message)
+	}
+	return proto.Unmarshal(data, protoMessage)
+}
+
+type pluggablePingServer struct {
+	pingv1connect.UnimplementedPingServiceHandler
+
+	ping    func(context.Context, *triple.Request) (*triple.Response, error)
+	sum     func(context.Context, *triple.ClientStream) (*triple.Response, error)
+	countUp func(context.Context, *triple.Request, *triple.ServerStream) error
+	cumSum  func(context.Context, *triple.BidiStream) error
+}
+
+func (p *pluggablePingServer) Ping(
+	ctx context.Context,
+	request *triple.Request,
+) (*triple.Response, error) {
+	return p.ping(ctx, request)
+}
+
+func (p *pluggablePingServer) Sum(
+	ctx context.Context,
+	stream *triple.ClientStream,
+) (*triple.Response, error) {
+	return p.sum(ctx, stream)
+}
+
+func (p *pluggablePingServer) CountUp(
+	ctx context.Context,
+	req *triple.Request,
+	stream *triple.ServerStream,
+) error {
+	return p.countUp(ctx, req, stream)
+}
+
+func (p *pluggablePingServer) CumSum(
+	ctx context.Context,
+	stream *triple.BidiStream,
+) error {
+	return p.cumSum(ctx, stream)
+}
+
+func failNoHTTP2(tb testing.TB, stream *triple.BidiStreamForClient) {
+	tb.Helper()
+	if err := stream.Send(&pingv1.CumSumRequest{}); err != nil {
+		assert.ErrorIs(tb, err, io.EOF)
+		assert.Equal(tb, triple.CodeOf(err), triple.CodeUnknown)
+	}
+	assert.Nil(tb, stream.CloseRequest())
+	err := stream.Receive(&pingv1.CumSumResponse{})
+	assert.NotNil(tb, err) // should be 505
+	assert.True(
+		tb,
+		strings.Contains(err.Error(), "HTTP status 505"),
+		assert.Sprintf("expected 505, got %v", err),
+	)
+	assert.Nil(tb, stream.CloseResponse())
+}
+
+func expectClientHeader(check bool, req triple.AnyRequest) error {
+	if !check {
+		return nil
+	}
+	if err := expectMetadata(req.Header(), "header", clientHeader, headerValue); err != nil {
+		return err
+	}
+	return nil
+}
+
+func expectMetadata(meta http.Header, metaType, key, value string) error {
+	if got := meta.Get(key); got != value {
+		return triple.NewError(triple.CodeInvalidArgument, fmt.Errorf(
+			"%s %q: got %q, expected %q",
+			metaType,
+			key,
+			got,
+			value,
+		))
+	}
+	return nil
+}
+
+type pingServer struct {
+	pingv1connect.UnimplementedPingServiceHandler
+
+	checkMetadata bool
+}
+
+func (p pingServer) Ping(ctx context.Context, request *triple.Request) (*triple.Response, error) {
+	if err := expectClientHeader(p.checkMetadata, request); err != nil {
+		return nil, err
+	}
+	if request.Peer().Addr == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	if request.Peer().Protocol == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer protocol"))
+	}
+	msg := request.Msg.(*pingv1.PingRequest)
+	response := triple.NewResponse(
+		&pingv1.PingResponse{
+			Number: msg.Number,
+			Text:   msg.Text,
+		},
+	)
+	response.Header().Set(handlerHeader, headerValue)
+	response.Trailer().Set(handlerTrailer, trailerValue)
+	return response, nil
+}
+
+func (p pingServer) Fail(ctx context.Context, request *triple.Request) (*triple.Response, error) {
+	if err := expectClientHeader(p.checkMetadata, request); err != nil {
+		return nil, err
+	}
+	if request.Peer().Addr == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	if request.Peer().Protocol == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer protocol"))
+	}
+	msg := request.Msg.(*pingv1.FailRequest)
+	err := triple.NewError(triple.Code(msg.Code), errors.New(errorMessage))
+	err.Meta().Set(handlerHeader, headerValue)
+	err.Meta().Set(handlerTrailer, trailerValue)
+	return nil, err
+}
+
+func (p pingServer) Sum(
+	ctx context.Context,
+	stream *triple.ClientStream,
+) (*triple.Response, error) {
+	if p.checkMetadata {
+		if err := expectMetadata(stream.RequestHeader(), "header", clientHeader, headerValue); err != nil {
+			return nil, err
+		}
+	}
+	if stream.Peer().Addr == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	if stream.Peer().Protocol == "" {
+		return nil, triple.NewError(triple.CodeInternal, errors.New("no peer protocol"))
+	}
+	var sum int64
+
+	for stream.Receive(&pingv1.SumRequest{}) {
+		msg := stream.Msg().(*pingv1.SumRequest)
+		sum += msg.Number
+	}
+	if stream.Err() != nil {
+		return nil, stream.Err()
+	}
+	response := triple.NewResponse(&pingv1.SumResponse{Sum: sum})
+	response.Header().Set(handlerHeader, headerValue)
+	response.Trailer().Set(handlerTrailer, trailerValue)
+	return response, nil
+}
+
+func (p pingServer) CountUp(
+	ctx context.Context,
+	request *triple.Request,
+	stream *triple.ServerStream,
+) error {
+	if err := expectClientHeader(p.checkMetadata, request); err != nil {
+		return err
+	}
+	if request.Peer().Addr == "" {
+		return triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	if request.Peer().Protocol == "" {
+		return triple.NewError(triple.CodeInternal, errors.New("no peer protocol"))
+	}
+	msg := request.Msg.(*pingv1.CountUpRequest)
+	if msg.Number <= 0 {
+		return triple.NewError(triple.CodeInvalidArgument, fmt.Errorf(
+			"number must be positive: got %v",
+			msg.Number,
+		))
+	}
+	stream.ResponseHeader().Set(handlerHeader, headerValue)
+	stream.ResponseTrailer().Set(handlerTrailer, trailerValue)
+	for i := int64(1); i <= msg.Number; i++ {
+		if err := stream.Send(&pingv1.CountUpResponse{Number: i}); err != nil {
+			return err
+		}
+	}
+	return nil
+}
+
+func (p pingServer) CumSum(
+	ctx context.Context,
+	stream *triple.BidiStream,
+) error {
+	var sum int64
+	if p.checkMetadata {
+		if err := expectMetadata(stream.RequestHeader(), "header", clientHeader, headerValue); err != nil {
+			return err
+		}
+	}
+	if stream.Peer().Addr == "" {
+		return triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	if stream.Peer().Protocol == "" {
+		return triple.NewError(triple.CodeInternal, errors.New("no peer address"))
+	}
+	stream.ResponseHeader().Set(handlerHeader, headerValue)
+	stream.ResponseTrailer().Set(handlerTrailer, trailerValue)
+	for {
+		msg := &pingv1.CumSumRequest{}
+		err := stream.Receive(msg)
+		if errors.Is(err, io.EOF) {
+			return nil
+		} else if err != nil {
+			return err
+		}
+		sum += msg.Number
+		if err := stream.Send(&pingv1.CumSumResponse{Sum: sum}); err != nil {
+			return err
+		}
+	}
+}
+
+type deflateReader struct {
+	r io.ReadCloser
+}
+
+func newDeflateReader(r io.Reader) *deflateReader {
+	return &deflateReader{r: flate.NewReader(r)}
+}
+
+func (d *deflateReader) Read(p []byte) (int, error) {
+	return d.r.Read(p)
+}
+
+func (d *deflateReader) Close() error {
+	return d.r.Close()
+}
+
+func (d *deflateReader) Reset(reader io.Reader) error {
+	if resetter, ok := d.r.(flate.Resetter); ok {
+		return resetter.Reset(reader, nil)
+	}
+	return fmt.Errorf("flate reader should implement flate.Resetter")
+}
+
+var _ triple.Decompressor = (*deflateReader)(nil)
+
+type trimTrailerWriter struct {
+	w http.ResponseWriter
+}
+
+func (l *trimTrailerWriter) Header() http.Header {
+	return l.w.Header()
+}
+
+// Write writes b to underlying writer and counts written size.
+func (l *trimTrailerWriter) Write(b []byte) (int, error) {
+	l.removeTrailers()
+	return l.w.Write(b)
+}
+
+// WriteHeader writes s to underlying writer and retains the status.
+func (l *trimTrailerWriter) WriteHeader(s int) {
+	l.removeTrailers()
+	l.w.WriteHeader(s)
+}
+
+// Flush implements http.Flusher.
+func (l *trimTrailerWriter) Flush() {
+	l.removeTrailers()
+	if f, ok := l.w.(http.Flusher); ok {
+		f.Flush()
+	}
+}
+
+func (l *trimTrailerWriter) removeTrailers() {
+	for _, v := range l.w.Header().Values("Trailer") {
+		l.w.Header().Del(v)
+	}
+	l.w.Header().Del("Trailer")
+	for k := range l.w.Header() {
+		if strings.HasPrefix(k, http.TrailerPrefix) {
+			l.w.Header().Del(k)
+		}
+	}
+}
+
+func newHTTPMiddlewareError() *triple.Error {
+	err := triple.NewError(triple.CodeResourceExhausted, errors.New("error from HTTP middleware"))
+	err.Meta().Set("Middleware-Foo", "bar")
+	return err
+}
+
+type failDecompressor struct {
+	triple.Decompressor
+}
+
+type failCompressor struct{}
+
+func (failCompressor) Write([]byte) (int, error) {
+	return 0, errors.New("failCompressor")
+}
+
+func (failCompressor) Close() error {
+	return errors.New("failCompressor")
+}
+
+func (failCompressor) Reset(io.Writer) {}
diff --git a/protocol/triple/triple_test.go b/protocol/triple/triple_test.go
new file mode 100644
index 0000000..11e4a73
--- /dev/null
+++ b/protocol/triple/triple_test.go
@@ -0,0 +1,290 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package triple
+
+import (
+	"context"
+	"errors"
+	"fmt"
+	"io"
+	"strings"
+	"testing"
+	"time"
+)
+
+import (
+	grpc_go "github.com/dubbogo/grpc-go"
+
+	"github.com/stretchr/testify/assert"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	invocation_impl "dubbo.apache.org/dubbo-go/v3/protocol/invocation"
+	dubbo3_api "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/dubbo3_server/api"
+	greet "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto"
+	dubbo3_greet "dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/dubbo3_gen"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/proto/triple_gen/greettriple"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/internal/server/api"
+	"dubbo.apache.org/dubbo-go/v3/protocol/triple/triple_protocol"
+	_ "dubbo.apache.org/dubbo-go/v3/proxy/proxy_factory"
+	"dubbo.apache.org/dubbo-go/v3/server"
+)
+
+const (
+	triplePort = "20000"
+	dubbo3Port = "20001"
+	listenAddr = "0.0.0.0"
+	name       = "triple"
+)
+
+type tripleInvoker struct {
+	url     *common.URL
+	info    *server.ServiceInfo
+	base    *protocol.BaseInvoker
+	handler interface{}
+}
+
+func (t *tripleInvoker) GetURL() *common.URL {
+	return t.url
+}
+
+func (t *tripleInvoker) IsAvailable() bool {
+	return t.base.IsAvailable()
+}
+
+func (t *tripleInvoker) Destroy() {
+	t.base.Destroy()
+}
+
+func (t *tripleInvoker) Invoke(ctx context.Context, invocation protocol.Invocation) protocol.Result {
+	name := invocation.MethodName()
+	args := invocation.Arguments()
+	// todo(DMwangnima): user map to represent Methods
+	for _, method := range t.info.Methods {
+		if method.Name == name {
+			res, err := method.MethodFunc(ctx, args, t.handler)
+			result := new(protocol.RPCResult)
+			result.SetResult(res)
+			result.SetError(err)
+			return result
+		}
+	}
+	panic(fmt.Sprintf("no match method for %s", name))
+}
+
+func runTripleServer(interfaceName string, addr string, info *server.ServiceInfo, handler interface{}) {
+	url := common.NewURLWithOptions(
+		common.WithPath(interfaceName),
+		common.WithLocation(addr),
+		common.WithPort(triplePort),
+	)
+	var invoker protocol.Invoker
+	if info != nil {
+		invoker = &tripleInvoker{
+			url:     url,
+			info:    info,
+			base:    protocol.NewBaseInvoker(url),
+			handler: handler,
+		}
+	}
+	GetProtocol().(*TripleProtocol).exportForTest(invoker, info)
+}
+
+func runOldTripleServer(addr string, desc *grpc_go.ServiceDesc) {
+	url := common.NewURLWithOptions(
+		common.WithPath(desc.ServiceName),
+		common.WithLocation(addr),
+		common.WithPort(dubbo3Port),
+		common.WithProtocol(TRIPLE),
+	)
+	srv := new(dubbo3_api.GreetDubbo3Server)
+	// todo(DMwangnima): add protocol config
+	config.SetRootConfig(
+		*config.NewRootConfigBuilder().
+			SetProvider(
+				config.NewProviderConfigBuilder().
+					AddService(common.GetReference(srv), config.NewServiceConfigBuilder().
+						SetInterface(desc.ServiceName).
+						Build()).
+					SetProxyFactory("default").
+					Build()).
+			Build())
+	config.SetProviderService(srv)
+	common.ServiceMap.Register(desc.ServiceName, TRIPLE, "", "", srv)
+	invoker := extension.GetProxyFactory("default").GetInvoker(url)
+	GetProtocol().(*TripleProtocol).exportForTest(invoker, nil)
+}
+
+func TestMain(m *testing.M) {
+	runTripleServer(
+		greettriple.GreetServiceName,
+		listenAddr,
+		&greettriple.GreetService_ServiceInfo,
+		new(api.GreetTripleServer),
+	)
+	runOldTripleServer(
+		listenAddr,
+		&dubbo3_greet.GreetService_ServiceDesc,
+	)
+	time.Sleep(3 * time.Second)
+	m.Run()
+}
+
+func TestInvoke(t *testing.T) {
+	invokeFunc := func(t *testing.T, port string, interfaceName string, methods []string) {
+		url := common.NewURLWithOptions(
+			common.WithInterface(interfaceName),
+			common.WithLocation("127.0.0.1"),
+			common.WithPort(port),
+			common.WithMethods(methods),
+			common.WithProtocol(TRIPLE),
+		)
+		invoker, err := NewTripleInvoker(url)
+		if err != nil {
+			t.Fatal(err)
+		}
+		tests := []struct {
+			desc       string
+			methodName string
+			params     []interface{}
+			invoke     func(t *testing.T, params []interface{}, res protocol.Result)
+			callType   string
+		}{
+			{
+				desc:       "Unary",
+				methodName: "Greet",
+				params: []interface{}{
+					&greet.GreetRequest{
+						Name: name,
+					},
+					&greet.GreetResponse{},
+				},
+				invoke: func(t *testing.T, params []interface{}, res protocol.Result) {
+					assert.Nil(t, res.Result())
+					assert.Nil(t, res.Error())
+					req := params[0].(*greet.GreetRequest)
+					resp := params[1].(*greet.GreetResponse)
+					assert.Equal(t, req.Name, resp.Greeting)
+				},
+				callType: constant.CallUnary,
+			},
+			{
+				desc:       "ClientStream",
+				methodName: "GreetClientStream",
+				invoke: func(t *testing.T, params []interface{}, res protocol.Result) {
+					assert.Nil(t, res.Error())
+					streamRaw, ok := res.Result().(*triple_protocol.ClientStreamForClient)
+					assert.True(t, ok)
+					stream := &greettriple.GreetServiceGreetClientStreamClient{ClientStreamForClient: streamRaw}
+
+					var expectRes []string
+					times := 5
+					for i := 1; i <= times; i++ {
+						expectRes = append(expectRes, name)
+						err := stream.Send(&greet.GreetClientStreamRequest{Name: name})
+						assert.Nil(t, err)
+					}
+					expectStr := strings.Join(expectRes, ",")
+					resp, err := stream.CloseAndRecv()
+					assert.Nil(t, err)
+					assert.Equal(t, expectStr, resp.Greeting)
+				},
+				callType: constant.CallClientStream,
+			},
+			{
+				desc:       "ServerStream",
+				methodName: "GreetServerStream",
+				params: []interface{}{
+					&greet.GreetServerStreamRequest{
+						Name: "dubbo",
+					},
+				},
+				invoke: func(t *testing.T, params []interface{}, res protocol.Result) {
+					assert.Nil(t, res.Error())
+					req := params[0].(*greet.GreetServerStreamRequest)
+					streamRaw, ok := res.Result().(*triple_protocol.ServerStreamForClient)
+					stream := &greettriple.GreetServiceGreetServerStreamClient{ServerStreamForClient: streamRaw}
+					assert.True(t, ok)
+					times := 5
+					for i := 1; i <= times; i++ {
+						for stream.Recv() {
+							assert.Nil(t, stream.Err())
+							assert.Equal(t, req.Name, stream.Msg().Greeting)
+						}
+						assert.True(t, true, errors.Is(stream.Err(), io.EOF))
+					}
+				},
+				callType: constant.CallServerStream,
+			},
+			{
+				desc:       "BidiStream",
+				methodName: "GreetStream",
+				invoke: func(t *testing.T, params []interface{}, res protocol.Result) {
+					assert.Nil(t, res.Error())
+					streamRaw, ok := res.Result().(*triple_protocol.BidiStreamForClient)
+					assert.True(t, ok)
+					stream := &greettriple.GreetServiceGreetStreamClient{BidiStreamForClient: streamRaw}
+					for i := 1; i <= 5; i++ {
+						err := stream.Send(&greet.GreetStreamRequest{Name: name})
+						assert.Nil(t, err)
+						resp, err := stream.Recv()
+						assert.Nil(t, err)
+						assert.Equal(t, name, resp.Greeting)
+					}
+					assert.Nil(t, stream.CloseRequest())
+					assert.Nil(t, stream.CloseResponse())
+				},
+				callType: constant.CallBidiStream,
+			},
+		}
+
+		for _, test := range tests {
+			t.Run(test.desc, func(t *testing.T) {
+				inv := invocation_impl.NewRPCInvocationWithOptions(
+					invocation_impl.WithMethodName(test.methodName),
+					// todo: process opts
+					invocation_impl.WithParameterRawValues(test.params),
+				)
+				inv.SetAttribute(constant.CallTypeKey, test.callType)
+				res := invoker.Invoke(context.Background(), inv)
+				test.invoke(t, test.params, res)
+			})
+		}
+	}
+	t.Run("invoke server code generated by triple", func(t *testing.T) {
+		invokeFunc(t, triplePort, greettriple.GreetService_ClientInfo.InterfaceName, greettriple.GreetService_ClientInfo.MethodNames)
+	})
+	t.Run("invoke server code generated by dubbo3", func(t *testing.T) {
+		desc := dubbo3_greet.GreetService_ServiceDesc
+		var methods []string
+		for _, method := range desc.Methods {
+			methods = append(methods, method.MethodName)
+		}
+		for _, stream := range desc.Streams {
+			methods = append(methods, stream.StreamName)
+		}
+
+		invokeFunc(t, dubbo3Port, desc.ServiceName, methods)
+	})
+
+}
diff --git a/registry/exposed_tmp/exposed.go b/registry/exposed_tmp/exposed.go
new file mode 100644
index 0000000..f6861e0
--- /dev/null
+++ b/registry/exposed_tmp/exposed.go
@@ -0,0 +1,132 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package exposed_tmp
+
+import (
+	"reflect"
+	"strconv"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+
+	perrors "github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+// RegisterServiceInstance register service instance
+func RegisterServiceInstance(applicationName string, tag string, metadataType string) {
+	url := selectMetadataServiceExportedURL()
+	if url == nil {
+		return
+	}
+	instance, err := createInstance(url, applicationName, tag, metadataType)
+	if err != nil {
+		panic(err)
+	}
+	p := extension.GetProtocol(constant.RegistryProtocol)
+	var rp registry.RegistryFactory
+	var ok bool
+	if rp, ok = p.(registry.RegistryFactory); !ok {
+		panic("dubbo registry protocol{" + reflect.TypeOf(p).String() + "} is invalid")
+	}
+	rs := rp.GetRegistries()
+	for _, r := range rs {
+		var sdr registry.ServiceDiscoveryHolder
+		if sdr, ok = r.(registry.ServiceDiscoveryHolder); !ok {
+			continue
+		}
+		// publish app level data to registry
+		logger.Infof("Starting register instance address %v", instance)
+		err := sdr.GetServiceDiscovery().Register(instance)
+		if err != nil {
+			panic(err)
+		}
+	}
+	// publish metadata to remote
+	if metadataType == constant.RemoteMetadataStorageType {
+		if remoteMetadataService, err := extension.GetRemoteMetadataService(); err == nil && remoteMetadataService != nil {
+			remoteMetadataService.PublishMetadata(applicationName)
+		}
+	}
+}
+
+// // nolint
+func createInstance(url *common.URL, applicationName string, tag string, metadataType string) (registry.ServiceInstance, error) {
+	port, err := strconv.ParseInt(url.Port, 10, 32)
+	if err != nil {
+		return nil, perrors.WithMessage(err, "invalid port: "+url.Port)
+	}
+
+	host := url.Ip
+	if len(host) == 0 {
+		host = common.GetLocalIp()
+	}
+
+	// usually we will add more metadata
+	metadata := make(map[string]string, 8)
+	metadata[constant.MetadataStorageTypePropertyName] = metadataType
+
+	instance := &registry.DefaultServiceInstance{
+		ServiceName: applicationName,
+		Host:        host,
+		Port:        int(port),
+		ID:          host + constant.KeySeparator + url.Port,
+		Enable:      true,
+		Healthy:     true,
+		Metadata:    metadata,
+		Tag:         tag,
+	}
+
+	for _, cus := range extension.GetCustomizers() {
+		cus.Customize(instance)
+	}
+
+	return instance, nil
+}
+
+// selectMetadataServiceExportedURL get already be exported url
+func selectMetadataServiceExportedURL() *common.URL {
+	var selectedUrl *common.URL
+	metaDataService, err := extension.GetLocalMetadataService(constant.DefaultKey)
+	if err != nil {
+		logger.Warnf("get metadata service exporter failed, pls check if you import _ \"dubbo.apache.org/dubbo-go/v3/metadata/service/local\"")
+		return nil
+	}
+	urlList, err := metaDataService.GetExportedURLs(constant.AnyValue, constant.AnyValue, constant.AnyValue, constant.AnyValue)
+	if err != nil {
+		panic(err)
+	}
+	if len(urlList) == 0 {
+		return nil
+	}
+	for _, url := range urlList {
+		selectedUrl = url
+		// rest first
+		if url.Protocol == "rest" {
+			break
+		}
+	}
+	return selectedUrl
+}
diff --git a/registry/options.go b/registry/options.go
new file mode 100644
index 0000000..af40112
--- /dev/null
+++ b/registry/options.go
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package registry
+
+import (
+	"time"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+type Options struct {
+	Registry *global.RegistryConfig
+}
+
+func DefaultOptions() *Options {
+	return &Options{Registry: global.DefaultRegistryConfig()}
+}
+
+type Option func(*Options)
+
+func WithEtcdV3() Option {
+	return func(opts *Options) {
+		// todo(DMwangnima): move etcdv3 to constant
+		opts.Registry.Protocol = "etcdv3"
+	}
+}
+
+func WithNacos() Option {
+	return func(opts *Options) {
+		opts.Registry.Protocol = constant.NacosKey
+	}
+}
+
+func WithPolaris() Option {
+	return func(opts *Options) {
+		opts.Registry.Protocol = constant.PolarisKey
+	}
+}
+
+func WithXDS() Option {
+	return func(opts *Options) {
+		opts.Registry.Protocol = constant.XDSRegistryKey
+	}
+}
+
+func WithZookeeper() Option {
+	return func(opts *Options) {
+		// todo(DMwangnima): move zookeeper to constant
+		opts.Registry.Protocol = "zookeeper"
+	}
+}
+
+func WithTimeout(timeout time.Duration) Option {
+	return func(opts *Options) {
+		opts.Registry.Timeout = timeout.String()
+	}
+}
+
+func WithGroup(group string) Option {
+	return func(opts *Options) {
+		opts.Registry.Group = group
+	}
+}
+
+func WithNamespace(namespace string) Option {
+	return func(opts *Options) {
+		opts.Registry.Namespace = namespace
+	}
+}
+
+func WithTTL(ttl time.Duration) Option {
+	return func(opts *Options) {
+		opts.Registry.TTL = ttl.String()
+	}
+}
+
+func WithAddress(address string) Option {
+	return func(opts *Options) {
+		opts.Registry.Address = address
+	}
+}
+
+func WithUsername(name string) Option {
+	return func(opts *Options) {
+		opts.Registry.Username = name
+	}
+}
+
+func WithPassword(password string) Option {
+	return func(opts *Options) {
+		opts.Registry.Password = password
+	}
+}
+
+func WithSimplified() Option {
+	return func(opts *Options) {
+		opts.Registry.Simplified = true
+	}
+}
+
+func WithPreferred() Option {
+	return func(opts *Options) {
+		opts.Registry.Preferred = true
+	}
+}
+
+func WithZone(zone string) Option {
+	return func(opts *Options) {
+		opts.Registry.Zone = zone
+	}
+}
+
+func WithWeight(weight int64) Option {
+	return func(opts *Options) {
+		opts.Registry.Weight = weight
+	}
+}
+
+func WithParams(params map[string]string) Option {
+	return func(opts *Options) {
+		opts.Registry.Params = params
+	}
+}
+
+func WithRegisterServiceAndInterface() Option {
+	return func(opts *Options) {
+		opts.Registry.RegistryType = constant.RegistryTypeAll
+	}
+}
+
+func WithRegisterInterface() Option {
+	return func(opts *Options) {
+		opts.Registry.RegistryType = constant.RegistryTypeInterface
+	}
+}
+
+func WithoutUseAsMetaReport() Option {
+	return func(opts *Options) {
+		opts.Registry.UseAsMetaReport = false
+	}
+}
+
+func WithoutUseAsConfigCenter() Option {
+	return func(opts *Options) {
+		opts.Registry.UseAsConfigCenter = false
+	}
+}
diff --git a/server/action.go b/server/action.go
new file mode 100644
index 0000000..6f93321
--- /dev/null
+++ b/server/action.go
@@ -0,0 +1,455 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package server
+
+import (
+	"container/list"
+	"fmt"
+	"net/url"
+	"os"
+	"strconv"
+	"strings"
+	"time"
+)
+
+import (
+	"github.com/dubbogo/gost/log/logger"
+	gxnet "github.com/dubbogo/gost/net"
+
+	perrors "github.com/pkg/errors"
+
+	"go.uber.org/atomic"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/common/extension"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+	"dubbo.apache.org/dubbo-go/v3/graceful_shutdown"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/protocol/protocolwrapper"
+)
+
+// Prefix returns dubbo.service.${InterfaceName}.
+func (svcOpts *ServiceOptions) Prefix() string {
+	return strings.Join([]string{constant.ServiceConfigPrefix, svcOpts.Id}, ".")
+}
+
+func (svcOpts *ServiceOptions) check() error {
+	srv := svcOpts.Service
+	// check if the limiter has been imported
+	if srv.TpsLimiter != "" {
+		_, err := extension.GetTpsLimiter(srv.TpsLimiter)
+		if err != nil {
+			panic(err)
+		}
+	}
+	if srv.TpsLimitStrategy != "" {
+		_, err := extension.GetTpsLimitStrategyCreator(srv.TpsLimitStrategy)
+		if err != nil {
+			panic(err)
+		}
+	}
+	if srv.TpsLimitRejectedHandler != "" {
+		_, err := extension.GetRejectedExecutionHandler(srv.TpsLimitRejectedHandler)
+		if err != nil {
+			panic(err)
+		}
+	}
+
+	if srv.TpsLimitInterval != "" {
+		tpsLimitInterval, err := strconv.ParseInt(srv.TpsLimitInterval, 0, 0)
+		if err != nil {
+			return fmt.Errorf("[ServiceConfig] Cannot parse the configuration tps.limit.interval for service %svcOpts, please check your configuration", srv.Interface)
+		}
+		if tpsLimitInterval < 0 {
+			return fmt.Errorf("[ServiceConfig] The configuration tps.limit.interval for service %svcOpts must be positive, please check your configuration", srv.Interface)
+		}
+	}
+
+	if srv.TpsLimitRate != "" {
+		tpsLimitRate, err := strconv.ParseInt(srv.TpsLimitRate, 0, 0)
+		if err != nil {
+			return fmt.Errorf("[ServiceConfig] Cannot parse the configuration tps.limit.rate for service %svcOpts, please check your configuration", srv.Interface)
+		}
+		if tpsLimitRate < 0 {
+			return fmt.Errorf("[ServiceConfig] The configuration tps.limit.rate for service %svcOpts must be positive, please check your configuration", srv.Interface)
+		}
+	}
+	return nil
+}
+
+// InitExported will set exported as false atom bool
+func (svcOpts *ServiceOptions) InitExported() {
+	svcOpts.exported = atomic.NewBool(false)
+}
+
+// IsExport will return whether the service config is exported or not
+func (svcOpts *ServiceOptions) IsExport() bool {
+	return svcOpts.exported.Load()
+}
+
+// Get Random Port
+func getRandomPort(protocolConfigs []*config.ProtocolConfig) *list.List {
+	ports := list.New()
+	for _, proto := range protocolConfigs {
+		if len(proto.Port) > 0 {
+			continue
+		}
+
+		tcp, err := gxnet.ListenOnTCPRandomPort(proto.Ip)
+		if err != nil {
+			panic(perrors.New(fmt.Sprintf("Get tcp port error, err is {%v}", err)))
+		}
+		defer tcp.Close()
+		ports.PushBack(strings.Split(tcp.Addr().String(), ":")[1])
+	}
+	return ports
+}
+
+func (svcOpts *ServiceOptions) ExportWithoutInfo() error {
+	return svcOpts.export(nil)
+}
+
+func (svcOpts *ServiceOptions) ExportWithInfo(info *ServiceInfo) error {
+	return svcOpts.export(info)
+}
+
+func (svcOpts *ServiceOptions) export(info *ServiceInfo) error {
+	srv := svcOpts.Service
+
+	if info != nil {
+		srv.Interface = info.InterfaceName
+		svcOpts.Id = info.InterfaceName
+		svcOpts.info = info
+	}
+	// TODO: delay needExport
+	if svcOpts.unexported != nil && svcOpts.unexported.Load() {
+		err := perrors.Errorf("The service %v has already unexported!", srv.Interface)
+		logger.Errorf(err.Error())
+		return err
+	}
+	if svcOpts.exported != nil && svcOpts.exported.Load() {
+		logger.Warnf("The service %v has already exported!", srv.Interface)
+		return nil
+	}
+
+	regUrls := make([]*common.URL, 0)
+	if !srv.NotRegister {
+		regUrls = config.LoadRegistries(srv.RegistryIDs, svcOpts.registriesCompat, common.PROVIDER)
+	}
+
+	urlMap := svcOpts.getUrlMap()
+	protocolConfigs := loadProtocol(srv.ProtocolIDs, svcOpts.protocolsCompat)
+	if len(protocolConfigs) == 0 {
+		logger.Warnf("The service %v'svcOpts '%v' protocols don't has right protocolConfigs, Please check your configuration center and transfer protocol ", srv.Interface, srv.ProtocolIDs)
+		return nil
+	}
+
+	var invoker protocol.Invoker
+	ports := getRandomPort(protocolConfigs)
+	nextPort := ports.Front()
+	proxyFactory := extension.GetProxyFactory(svcOpts.ProxyFactoryKey)
+	for _, proto := range protocolConfigs {
+		// *important* Register should have been replaced by processing of ServiceInfo.
+		// but many modules like metadata need to make use of information from ServiceMap.
+		// todo(DMwangnimg): finish replacing procedure
+
+		// registry the service reflect
+		methods, err := common.ServiceMap.Register(srv.Interface, proto.Name, srv.Group, srv.Version, svcOpts.rpcService)
+		if err != nil {
+			formatErr := perrors.Errorf("The service %v needExport the protocol %v error! Error message is %v.",
+				srv.Interface, proto.Name, err.Error())
+			logger.Errorf(formatErr.Error())
+			return formatErr
+		}
+
+		port := proto.Port
+		if len(proto.Port) == 0 {
+			port = nextPort.Value.(string)
+			nextPort = nextPort.Next()
+		}
+		ivkURL := common.NewURLWithOptions(
+			common.WithPath(srv.Interface),
+			common.WithProtocol(proto.Name),
+			common.WithIp(proto.Ip),
+			common.WithPort(port),
+			common.WithParams(urlMap),
+			common.WithParamsValue(constant.BeanNameKey, svcOpts.Id),
+			//common.WithParamsValue(constant.SslEnabledKey, strconv.FormatBool(config.GetSslEnabled())),
+			common.WithMethods(strings.Split(methods, ",")),
+			// todo(DMwangnima): remove this
+			common.WithAttribute(constant.ServiceInfoKey, info),
+			common.WithToken(srv.Token),
+			common.WithParamsValue(constant.MetadataTypeKey, svcOpts.metadataType),
+			// fix https://github.com/apache/dubbo-go/issues/2176
+			common.WithParamsValue(constant.MaxServerSendMsgSize, proto.MaxServerSendMsgSize),
+			common.WithParamsValue(constant.MaxServerRecvMsgSize, proto.MaxServerRecvMsgSize),
+		)
+		if len(srv.Tag) > 0 {
+			ivkURL.AddParam(constant.Tagkey, srv.Tag)
+		}
+
+		// post process the URL to be exported
+		svcOpts.postProcessConfig(ivkURL)
+		// config post processor may set "needExport" to false
+		if !ivkURL.GetParamBool(constant.ExportKey, true) {
+			return nil
+		}
+
+		if len(regUrls) > 0 {
+			svcOpts.cacheMutex.Lock()
+			if svcOpts.cacheProtocol == nil {
+				logger.Debugf(fmt.Sprintf("First load the registry protocol, url is {%v}!", ivkURL))
+				svcOpts.cacheProtocol = extension.GetProtocol(constant.RegistryProtocol)
+			}
+			svcOpts.cacheMutex.Unlock()
+
+			for _, regUrl := range regUrls {
+				setRegistrySubURL(ivkURL, regUrl)
+				if info == nil {
+					invoker = proxyFactory.GetInvoker(regUrl)
+				} else {
+					invoker = newInfoInvoker(regUrl, info, svcOpts.rpcService)
+				}
+				exporter := svcOpts.cacheProtocol.Export(invoker)
+				if exporter == nil {
+					return perrors.New(fmt.Sprintf("Registry protocol new exporter error, registry is {%v}, url is {%v}", regUrl, ivkURL))
+				}
+				svcOpts.exporters = append(svcOpts.exporters, exporter)
+			}
+		} else {
+			if ivkURL.GetParam(constant.InterfaceKey, "") == constant.MetadataServiceName {
+				ms, err := extension.GetLocalMetadataService("")
+				if err != nil {
+					logger.Warnf("needExport org.apache.dubbo.metadata.MetadataService failed beacause of %svcOpts ! pls check if you import _ \"dubbo.apache.org/dubbo-go/v3/metadata/service/local\"", err)
+					return nil
+				}
+				if err := ms.SetMetadataServiceURL(ivkURL); err != nil {
+					logger.Warnf("SetMetadataServiceURL error = %svcOpts", err)
+				}
+			}
+			if info == nil {
+				invoker = proxyFactory.GetInvoker(ivkURL)
+			} else {
+				invoker = newInfoInvoker(ivkURL, info, svcOpts.rpcService)
+			}
+			exporter := extension.GetProtocol(protocolwrapper.FILTER).Export(invoker)
+			if exporter == nil {
+				return perrors.New(fmt.Sprintf("Filter protocol without registry new exporter error, url is {%v}", ivkURL))
+			}
+			svcOpts.exporters = append(svcOpts.exporters, exporter)
+		}
+		publishServiceDefinition(ivkURL)
+		// this protocol would be destroyed in graceful_shutdown
+		// please refer to (https://github.com/apache/dubbo-go/issues/2429)
+		graceful_shutdown.RegisterProtocol(proto.Name)
+	}
+	svcOpts.exported.Store(true)
+	return nil
+}
+
+// setRegistrySubURL set registry sub url is ivkURl
+func setRegistrySubURL(ivkURL *common.URL, regUrl *common.URL) {
+	ivkURL.AddParam(constant.RegistryKey, regUrl.GetParam(constant.RegistryKey, ""))
+	regUrl.SubURL = ivkURL
+}
+
+// loadProtocol filter protocols by ids
+func loadProtocol(protocolIds []string, protocols map[string]*config.ProtocolConfig) []*config.ProtocolConfig {
+	returnProtocols := make([]*config.ProtocolConfig, 0, len(protocols))
+	for _, v := range protocolIds {
+		for k, config := range protocols {
+			if v == k {
+				returnProtocols = append(returnProtocols, config)
+			}
+		}
+	}
+	return returnProtocols
+}
+
+// Unexport will call unexport of all exporters service config exported
+func (svcOpts *ServiceOptions) Unexport() {
+	if !svcOpts.exported.Load() {
+		return
+	}
+	if svcOpts.unexported.Load() {
+		return
+	}
+
+	func() {
+		svcOpts.exportersLock.Lock()
+		defer svcOpts.exportersLock.Unlock()
+		for _, exporter := range svcOpts.exporters {
+			exporter.UnExport()
+		}
+		svcOpts.exporters = nil
+	}()
+
+	svcOpts.exported.Store(false)
+	svcOpts.unexported.Store(true)
+}
+
+// Implement only store the @s and return
+func (svcOpts *ServiceOptions) Implement(rpcService common.RPCService) {
+	svcOpts.rpcService = rpcService
+}
+
+func (svcOpts *ServiceOptions) getUrlMap() url.Values {
+	srv := svcOpts.Service
+	app := svcOpts.applicationCompat
+
+	urlMap := url.Values{}
+	// first set user params
+	for k, v := range srv.Params {
+		urlMap.Set(k, v)
+	}
+	urlMap.Set(constant.InterfaceKey, srv.Interface)
+	urlMap.Set(constant.TimestampKey, strconv.FormatInt(time.Now().Unix(), 10))
+	urlMap.Set(constant.ClusterKey, srv.Cluster)
+	urlMap.Set(constant.LoadbalanceKey, srv.Loadbalance)
+	urlMap.Set(constant.WarmupKey, srv.Warmup)
+	urlMap.Set(constant.RetriesKey, srv.Retries)
+	if srv.Group != "" {
+		urlMap.Set(constant.GroupKey, srv.Group)
+	}
+	if srv.Version != "" {
+		urlMap.Set(constant.VersionKey, srv.Version)
+	}
+	urlMap.Set(constant.RegistryRoleKey, strconv.Itoa(common.PROVIDER))
+	urlMap.Set(constant.ReleaseKey, "dubbo-golang-"+constant.Version)
+	urlMap.Set(constant.SideKey, (common.RoleType(common.PROVIDER)).Role())
+	// todo: move
+	urlMap.Set(constant.SerializationKey, srv.Serialization)
+	// application config info
+	urlMap.Set(constant.ApplicationKey, app.Name)
+	urlMap.Set(constant.OrganizationKey, app.Organization)
+	urlMap.Set(constant.NameKey, app.Name)
+	urlMap.Set(constant.ModuleKey, app.Module)
+	urlMap.Set(constant.AppVersionKey, app.Version)
+	urlMap.Set(constant.OwnerKey, app.Owner)
+	urlMap.Set(constant.EnvironmentKey, app.Environment)
+
+	//filter
+	var filters string
+	if srv.Filter == "" {
+		filters = constant.DefaultServiceFilters
+	} else {
+		filters = srv.Filter
+	}
+	if svcOpts.adaptiveService {
+		filters += fmt.Sprintf(",%svcOpts", constant.AdaptiveServiceProviderFilterKey)
+	}
+	urlMap.Set(constant.ServiceFilterKey, filters)
+
+	// filter special config
+	urlMap.Set(constant.AccessLogFilterKey, srv.AccessLog)
+	// tps limiter
+	urlMap.Set(constant.TPSLimitStrategyKey, srv.TpsLimitStrategy)
+	urlMap.Set(constant.TPSLimitIntervalKey, srv.TpsLimitInterval)
+	urlMap.Set(constant.TPSLimitRateKey, srv.TpsLimitRate)
+	urlMap.Set(constant.TPSLimiterKey, srv.TpsLimiter)
+	urlMap.Set(constant.TPSRejectedExecutionHandlerKey, srv.TpsLimitRejectedHandler)
+	urlMap.Set(constant.TracingConfigKey, srv.TracingKey)
+
+	// execute limit filter
+	urlMap.Set(constant.ExecuteLimitKey, srv.ExecuteLimit)
+	urlMap.Set(constant.ExecuteRejectedExecutionHandlerKey, srv.ExecuteLimitRejectedHandler)
+
+	// auth filter
+	urlMap.Set(constant.ServiceAuthKey, srv.Auth)
+	urlMap.Set(constant.ParameterSignatureEnableKey, srv.ParamSign)
+
+	// whether to needExport or not
+	urlMap.Set(constant.ExportKey, strconv.FormatBool(svcOpts.needExport))
+	urlMap.Set(constant.PIDKey, fmt.Sprintf("%d", os.Getpid()))
+
+	for _, v := range srv.Methods {
+		prefix := "methods." + v.Name + "."
+		urlMap.Set(prefix+constant.LoadbalanceKey, v.LoadBalance)
+		urlMap.Set(prefix+constant.RetriesKey, v.Retries)
+		urlMap.Set(prefix+constant.WeightKey, strconv.FormatInt(v.Weight, 10))
+
+		urlMap.Set(prefix+constant.TPSLimitStrategyKey, v.TpsLimitStrategy)
+		urlMap.Set(prefix+constant.TPSLimitIntervalKey, v.TpsLimitInterval)
+		urlMap.Set(prefix+constant.TPSLimitRateKey, v.TpsLimitRate)
+
+		urlMap.Set(constant.ExecuteLimitKey, v.ExecuteLimit)
+		urlMap.Set(constant.ExecuteRejectedExecutionHandlerKey, v.ExecuteLimitRejectedHandler)
+	}
+
+	return urlMap
+}
+
+// GetExportedUrls will return the url in service config's exporter
+func (svcOpts *ServiceOptions) GetExportedUrls() []*common.URL {
+	if svcOpts.exported.Load() {
+		var urls []*common.URL
+		for _, exporter := range svcOpts.exporters {
+			urls = append(urls, exporter.GetInvoker().GetURL())
+		}
+		return urls
+	}
+	return nil
+}
+
+// postProcessConfig asks registered ConfigPostProcessor to post-process the current ServiceConfig.
+func (svcOpts *ServiceOptions) postProcessConfig(url *common.URL) {
+	for _, p := range extension.GetConfigPostProcessors() {
+		p.PostProcessServiceConfig(url)
+	}
+}
+
+func publishServiceDefinition(url *common.URL) {
+	localService, err := extension.GetLocalMetadataService(constant.DefaultKey)
+	if err != nil {
+		logger.Warnf("get local metadata service failed, please check if you have imported _ \"dubbo.apache.org/dubbo-go/v3/metadata/service/local\"")
+		return
+	}
+	localService.PublishServiceDefinition(url)
+	if url.GetParam(constant.MetadataTypeKey, "") != constant.RemoteMetadataStorageType {
+		return
+	}
+	if remoteMetadataService, err := extension.GetRemoteMetadataService(); err == nil && remoteMetadataService != nil {
+		remoteMetadataService.PublishServiceDefinition(url)
+	}
+}
+
+// todo(DMwangnima): think about moving this function to a common place(e.g. /common/config)
+func getRegistryIds(registries map[string]*global.RegistryConfig) []string {
+	ids := make([]string, 0)
+	for key := range registries {
+		ids = append(ids, key)
+	}
+	return removeDuplicateElement(ids)
+}
+
+// removeDuplicateElement remove duplicate element
+func removeDuplicateElement(items []string) []string {
+	result := make([]string, 0, len(items))
+	temp := map[string]struct{}{}
+	for _, item := range items {
+		if _, ok := temp[item]; !ok && item != "" {
+			temp[item] = struct{}{}
+			result = append(result, item)
+		}
+	}
+	return result
+}
diff --git a/server/compat.go b/server/compat.go
new file mode 100644
index 0000000..9e34254
--- /dev/null
+++ b/server/compat.go
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package server
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/config"
+	"dubbo.apache.org/dubbo-go/v3/global"
+)
+
+// these functions are used to resolve circular dependencies temporarily.
+// please refer to issue(https://github.com/apache/dubbo-go/issues/2377)
+// todo(DMwangnima): remove these functions when refactoring dubbo-go
+func compatApplicationConfig(c *global.ApplicationConfig) *config.ApplicationConfig {
+	return &config.ApplicationConfig{
+		Organization: c.Organization,
+		Name:         c.Name,
+		Module:       c.Module,
+		Group:        c.Group,
+		Version:      c.Version,
+		Owner:        c.Owner,
+		Environment:  c.Environment,
+		MetadataType: c.MetadataType,
+		Tag:          c.Tag,
+	}
+}
+
+func compatRegistryConfig(c *global.RegistryConfig) *config.RegistryConfig {
+	return &config.RegistryConfig{
+		Protocol:          c.Protocol,
+		Timeout:           c.Timeout,
+		Group:             c.Group,
+		Namespace:         c.Namespace,
+		TTL:               c.TTL,
+		Address:           c.Address,
+		Username:          c.Username,
+		Password:          c.Password,
+		Simplified:        c.Simplified,
+		Preferred:         c.Preferred,
+		Zone:              c.Zone,
+		Weight:            c.Weight,
+		Params:            c.Params,
+		RegistryType:      c.RegistryType,
+		UseAsMetaReport:   c.UseAsMetaReport,
+		UseAsConfigCenter: c.UseAsConfigCenter,
+	}
+}
+
+func compatMethodConfig(c *global.MethodConfig) *config.MethodConfig {
+	return &config.MethodConfig{
+		InterfaceId:                 c.InterfaceId,
+		InterfaceName:               c.InterfaceName,
+		Name:                        c.Name,
+		Retries:                     c.Retries,
+		LoadBalance:                 c.LoadBalance,
+		Weight:                      c.Weight,
+		TpsLimitInterval:            c.TpsLimitInterval,
+		TpsLimitRate:                c.TpsLimitRate,
+		TpsLimitStrategy:            c.TpsLimitStrategy,
+		ExecuteLimit:                c.ExecuteLimit,
+		ExecuteLimitRejectedHandler: c.ExecuteLimitRejectedHandler,
+		Sticky:                      c.Sticky,
+		RequestTimeout:              c.RequestTimeout,
+	}
+}
+
+func compatProtocolConfig(c *global.ProtocolConfig) *config.ProtocolConfig {
+	return &config.ProtocolConfig{
+		Name:                 c.Name,
+		Ip:                   c.Ip,
+		Port:                 c.Port,
+		Params:               c.Params,
+		MaxServerSendMsgSize: c.MaxServerSendMsgSize,
+		MaxServerRecvMsgSize: c.MaxServerRecvMsgSize,
+	}
+}
diff --git a/server/options.go b/server/options.go
new file mode 100644
index 0000000..6dff299
--- /dev/null
+++ b/server/options.go
@@ -0,0 +1,551 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package server
+
+import (
+	"strconv"
+	"sync"
+	"time"
+)
+
+import (
+	"github.com/creasty/defaults"
+
+	"github.com/dubbogo/gost/log/logger"
+
+	perrors "github.com/pkg/errors"
+
+	"go.uber.org/atomic"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	commonCfg "dubbo.apache.org/dubbo-go/v3/common/config"
+	"dubbo.apache.org/dubbo-go/v3/common/constant"
+	"dubbo.apache.org/dubbo-go/v3/config"
+	aslimiter "dubbo.apache.org/dubbo-go/v3/filter/adaptivesvc/limiter"
+	"dubbo.apache.org/dubbo-go/v3/global"
+	"dubbo.apache.org/dubbo-go/v3/graceful_shutdown"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	"dubbo.apache.org/dubbo-go/v3/registry"
+)
+
+type ServerOptions struct {
+	Provider    *global.ProviderConfig
+	Application *global.ApplicationConfig
+	Registries  map[string]*global.RegistryConfig
+	Protocols   map[string]*global.ProtocolConfig
+	Tracings    map[string]*global.TracingConfig
+	Shutdown    *global.ShutdownConfig
+
+	providerCompat *config.ProviderConfig
+}
+
+func defaultServerOptions() *ServerOptions {
+	return &ServerOptions{
+		Application: global.DefaultApplicationConfig(),
+		Provider:    global.DefaultProviderConfig(),
+		Shutdown:    global.DefaultShutdownConfig(),
+	}
+}
+
+// todo(DMwangnima): think about the timing to initialize Registry, Protocol, Tracing
+func (srvOpts *ServerOptions) init(opts ...ServerOption) error {
+	for _, opt := range opts {
+		opt(srvOpts)
+	}
+	if err := defaults.Set(srvOpts); err != nil {
+		return err
+	}
+
+	prov := srvOpts.Provider
+
+	prov.RegistryIDs = commonCfg.TranslateIds(prov.RegistryIDs)
+	if len(prov.RegistryIDs) <= 0 {
+		prov.RegistryIDs = getRegistryIds(srvOpts.Registries)
+	}
+
+	prov.ProtocolIDs = commonCfg.TranslateIds(prov.ProtocolIDs)
+
+	if prov.TracingKey == "" && len(srvOpts.Tracings) > 0 {
+		for key := range srvOpts.Tracings {
+			prov.TracingKey = key
+			break
+		}
+	}
+
+	if err := commonCfg.Verify(prov); err != nil {
+		return err
+	}
+
+	// enable adaptive service verbose
+	if prov.AdaptiveServiceVerbose {
+		if !prov.AdaptiveService {
+			return perrors.Errorf("The adaptive service is disabled, " +
+				"adaptive service verbose should be disabled either.")
+		}
+		logger.Infof("adaptive service verbose is enabled.")
+		logger.Debugf("debug-level info could be shown.")
+		aslimiter.Verbose = true
+	}
+
+	// init graceful_shutdown
+	graceful_shutdown.Init(graceful_shutdown.WithShutdown_Config(srvOpts.Shutdown))
+
+	return nil
+}
+
+type ServerOption func(*ServerOptions)
+
+// ---------- For user ----------
+
+// todo(DMwangnima): change Filter Option like Cluster and LoadBalance
+func WithServer_Filter(filter string) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.Filter = filter
+	}
+}
+
+// todo(DMwangnima): think about a more ideal configuration style
+func WithServer_RegistryIDs(registryIDs []string) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.RegistryIDs = registryIDs
+	}
+}
+
+func WithServer_Registry(key string, opts ...registry.Option) ServerOption {
+	regOpts := registry.DefaultOptions()
+	for _, opt := range opts {
+		opt(regOpts)
+	}
+
+	return func(srvOpts *ServerOptions) {
+		if srvOpts.Registries == nil {
+			srvOpts.Registries = make(map[string]*global.RegistryConfig)
+		}
+		srvOpts.Registries[key] = regOpts.Registry
+	}
+}
+
+// todo(DMwangnima): think about a more ideal configuration style
+func WithServer_ProtocolIDs(protocolIDs []string) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.ProtocolIDs = protocolIDs
+	}
+}
+
+func WithServer_Protocol(key string, opts ...protocol.Option) ServerOption {
+	proOpts := protocol.DefaultOptions()
+	for _, opt := range opts {
+		opt(proOpts)
+	}
+
+	return func(srvOpts *ServerOptions) {
+		if srvOpts.Protocols == nil {
+			srvOpts.Protocols = make(map[string]*global.ProtocolConfig)
+		}
+		srvOpts.Protocols[key] = proOpts.Protocol
+	}
+}
+
+func WithServer_TracingKey(tracingKey string) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.TracingKey = tracingKey
+	}
+}
+
+// todo(DMwangnima): this configuration would be used by filter/hystrix
+// think about a more ideal way to configure
+func WithServer_FilterConf(conf interface{}) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.FilterConf = conf
+	}
+}
+
+func WithServer_AdaptiveService() ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.AdaptiveService = true
+	}
+}
+
+func WithServer_AdaptiveServiceVerbose() ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Provider.AdaptiveServiceVerbose = true
+	}
+}
+
+// ========== For framework ==========
+// These functions should not be invoked by users
+
+func SetServer_Application(application *global.ApplicationConfig) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Application = application
+	}
+}
+
+func SetServer_Registries(regs map[string]*global.RegistryConfig) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Registries = regs
+	}
+}
+
+func SetServer_Protocols(pros map[string]*global.ProtocolConfig) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Protocols = pros
+	}
+}
+
+func SetServer_Tracings(tras map[string]*global.TracingConfig) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Tracings = tras
+	}
+}
+
+func SetServer_Shutdown(shutdown *global.ShutdownConfig) ServerOption {
+	return func(opts *ServerOptions) {
+		opts.Shutdown = shutdown
+	}
+}
+
+type ServiceOptions struct {
+	Application *global.ApplicationConfig
+	Provider    *global.ProviderConfig
+	Service     *global.ServiceConfig
+	Registries  map[string]*global.RegistryConfig
+	Protocols   map[string]*global.ProtocolConfig
+
+	Id              string
+	unexported      *atomic.Bool
+	exported        *atomic.Bool
+	needExport      bool
+	metadataType    string
+	info            *ServiceInfo
+	ProxyFactoryKey string
+	rpcService      common.RPCService
+	cacheMutex      sync.Mutex
+	cacheProtocol   protocol.Protocol
+	exportersLock   sync.Mutex
+	exporters       []protocol.Exporter
+	adaptiveService bool
+
+	methodsCompat     []*config.MethodConfig
+	applicationCompat *config.ApplicationConfig
+	registriesCompat  map[string]*config.RegistryConfig
+	protocolsCompat   map[string]*config.ProtocolConfig
+}
+
+func defaultServiceOptions() *ServiceOptions {
+	return &ServiceOptions{
+		Service:     global.DefaultServiceConfig(),
+		Application: global.DefaultApplicationConfig(),
+		unexported:  atomic.NewBool(false),
+		exported:    atomic.NewBool(false),
+		needExport:  true,
+	}
+}
+
+func (svcOpts *ServiceOptions) init(opts ...ServiceOption) error {
+	for _, opt := range opts {
+		opt(svcOpts)
+	}
+	if err := defaults.Set(svcOpts); err != nil {
+		return err
+	}
+
+	srv := svcOpts.Service
+
+	svcOpts.exported = atomic.NewBool(false)
+
+	application := svcOpts.Application
+	if application != nil {
+		svcOpts.applicationCompat = compatApplicationConfig(application)
+		if err := svcOpts.applicationCompat.Init(); err != nil {
+			return err
+		}
+		// todo(DMwangnima): make this clearer
+		// this statement is responsible for setting rootConfig.Application
+		// since many modules would retrieve this information directly.
+		config.GetRootConfig().Application = svcOpts.applicationCompat
+		svcOpts.metadataType = svcOpts.applicationCompat.MetadataType
+		if srv.Group == "" {
+			srv.Group = svcOpts.applicationCompat.Group
+		}
+		if srv.Version == "" {
+			srv.Version = svcOpts.applicationCompat.Version
+		}
+	}
+	svcOpts.unexported = atomic.NewBool(false)
+
+	// initialize Registries
+	if len(srv.RCRegistriesMap) == 0 {
+		srv.RCRegistriesMap = svcOpts.Registries
+	}
+	if len(srv.RCRegistriesMap) > 0 {
+		svcOpts.registriesCompat = make(map[string]*config.RegistryConfig)
+		for key, reg := range srv.RCRegistriesMap {
+			svcOpts.registriesCompat[key] = compatRegistryConfig(reg)
+			if err := svcOpts.registriesCompat[key].Init(); err != nil {
+				return err
+			}
+		}
+	}
+
+	// initialize Protocols
+	if len(srv.RCProtocolsMap) == 0 {
+		srv.RCProtocolsMap = svcOpts.Protocols
+	}
+	if len(srv.RCProtocolsMap) > 0 {
+		svcOpts.protocolsCompat = make(map[string]*config.ProtocolConfig)
+		for key, pro := range srv.RCProtocolsMap {
+			svcOpts.protocolsCompat[key] = compatProtocolConfig(pro)
+			if err := svcOpts.protocolsCompat[key].Init(); err != nil {
+				return err
+			}
+		}
+	}
+
+	srv.RegistryIDs = commonCfg.TranslateIds(srv.RegistryIDs)
+	if len(srv.RegistryIDs) <= 0 {
+		srv.RegistryIDs = svcOpts.Provider.RegistryIDs
+	}
+	if srv.RegistryIDs == nil || len(srv.RegistryIDs) <= 0 {
+		srv.NotRegister = true
+	}
+
+	srv.ProtocolIDs = commonCfg.TranslateIds(srv.ProtocolIDs)
+	if len(srv.ProtocolIDs) <= 0 {
+		srv.ProtocolIDs = svcOpts.Provider.ProtocolIDs
+	}
+	if len(srv.ProtocolIDs) <= 0 {
+		for name := range svcOpts.Protocols {
+			srv.ProtocolIDs = append(srv.ProtocolIDs, name)
+		}
+	}
+
+	if srv.TracingKey == "" {
+		srv.TracingKey = svcOpts.Provider.TracingKey
+	}
+
+	err := svcOpts.check()
+	if err != nil {
+		panic(err)
+	}
+	svcOpts.needExport = true
+	return commonCfg.Verify(svcOpts)
+}
+
+type ServiceOption func(*ServiceOptions)
+
+// ---------- For user ----------
+
+// todo(DMwangnima): think about a more ideal configuration style
+func WithRegistryIDs(registryIDs []string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		if len(registryIDs) <= 0 {
+			cfg.Service.RegistryIDs = registryIDs
+		}
+	}
+}
+
+// todo(DMwangnima): change Filter Option like Cluster and LoadBalance
+func WithFilter(filter string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		cfg.Service.Filter = filter
+	}
+}
+
+// todo(DMwangnima): think about a more ideal configuration style
+func WithProtocolIDs(protocolIDs []string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		if len(protocolIDs) <= 0 {
+			cfg.Service.ProtocolIDs = protocolIDs
+		}
+	}
+}
+
+func WithTracingKey(tracingKey string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		cfg.Service.TracingKey = tracingKey
+	}
+}
+
+// ========== LoadBalance Strategy ==========
+
+func WithLoadBalanceConsistentHashing() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyConsistentHashing
+	}
+}
+
+func WithLoadBalanceLeastActive() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyLeastActive
+	}
+}
+
+func WithLoadBalanceRandom() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyRandom
+	}
+}
+
+func WithLoadBalanceRoundRobin() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyRoundRobin
+	}
+}
+
+func WithLoadBalanceP2C() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyP2C
+	}
+}
+
+func WithLoadBalanceXDSRingHash() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Loadbalance = constant.LoadBalanceKeyLeastActive
+	}
+}
+
+// warmUp is in seconds
+func WithWarmUp(warmUp time.Duration) ServiceOption {
+	return func(opts *ServiceOptions) {
+		warmUpSec := int(warmUp / time.Second)
+		opts.Service.Warmup = strconv.Itoa(warmUpSec)
+	}
+}
+
+// ========== Cluster Strategy ==========
+
+func WithClusterAvailable() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyAvailable
+	}
+}
+
+func WithClusterBroadcast() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyBroadcast
+	}
+}
+
+func WithClusterFailBack() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyFailback
+	}
+}
+
+func WithClusterFailFast() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyFailfast
+	}
+}
+
+func WithClusterFailOver() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyFailover
+	}
+}
+
+func WithClusterFailSafe() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyFailsafe
+	}
+}
+
+func WithClusterForking() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyForking
+	}
+}
+
+func WithClusterZoneAware() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyZoneAware
+	}
+}
+
+func WithClusterAdaptiveService() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Cluster = constant.ClusterKeyAdaptiveService
+	}
+}
+
+func WithGroup(group string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		cfg.Service.Group = group
+	}
+}
+
+func WithVersion(version string) ServiceOption {
+	return func(cfg *ServiceOptions) {
+		cfg.Service.Version = version
+	}
+}
+
+func WithJSON() ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Serialization = constant.JSONSerialization
+	}
+}
+
+// WithToken should be used with WithFilter("token")
+func WithToken(token string) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service.Token = token
+	}
+}
+
+func WithNotRegister() ServiceOption {
+	return func(cfg *ServiceOptions) {
+		cfg.Service.NotRegister = true
+	}
+}
+
+// ----------For framework----------
+// These functions should not be invoked by users
+
+func SetApplication(application *global.ApplicationConfig) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Application = application
+	}
+}
+
+func SetProvider(provider *global.ProviderConfig) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Provider = provider
+	}
+}
+
+func SetService(service *global.ServiceConfig) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Service = service
+	}
+}
+
+func SetRegistries(regs map[string]*global.RegistryConfig) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Registries = regs
+	}
+}
+
+func SetProtocols(pros map[string]*global.ProtocolConfig) ServiceOption {
+	return func(opts *ServiceOptions) {
+		opts.Protocols = pros
+	}
+}
diff --git a/server/server.go b/server/server.go
new file mode 100644
index 0000000..3cbcd84
--- /dev/null
+++ b/server/server.go
@@ -0,0 +1,179 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package server
+
+import (
+	"context"
+	"fmt"
+)
+
+import (
+	"github.com/pkg/errors"
+)
+
+import (
+	"dubbo.apache.org/dubbo-go/v3/common"
+	"dubbo.apache.org/dubbo-go/v3/metadata"
+	"dubbo.apache.org/dubbo-go/v3/protocol"
+	registry_exposed "dubbo.apache.org/dubbo-go/v3/registry/exposed_tmp"
+)
+
+type Server struct {
+	invoker protocol.Invoker
+	info    *ServiceInfo
+
+	cfg *ServerOptions
+}
+
+// ServiceInfo is meta info of a service
+type ServiceInfo struct {
+	InterfaceName string
+	ServiceType   interface{}
+	Methods       []MethodInfo
+	Meta          map[string]interface{}
+}
+
+type infoInvoker struct {
+	url       *common.URL
+	base      *protocol.BaseInvoker
+	info      *ServiceInfo
+	svc       common.RPCService
+	methodMap map[string]*MethodInfo
+}
+
+func (ii *infoInvoker) init() {
+	url := ii.base.GetURL()
+	if url.SubURL != nil {
+		url = url.SubURL
+	}
+	ii.url = url
+	methodMap := make(map[string]*MethodInfo)
+	for i := range ii.info.Methods {
+		methodMap[ii.info.Methods[i].Name] = &ii.info.Methods[i]
+	}
+	ii.methodMap = methodMap
+}
+
+func (ii *infoInvoker) GetURL() *common.URL {
+	return ii.base.GetURL()
+}
+
+func (ii *infoInvoker) IsAvailable() bool {
+	return ii.base.IsAvailable()
+}
+
+func (ii *infoInvoker) Destroy() {
+	ii.base.Destroy()
+}
+
+func (ii *infoInvoker) Invoke(ctx context.Context, invocation protocol.Invocation) protocol.Result {
+	name := invocation.MethodName()
+	args := invocation.Arguments()
+	result := new(protocol.RPCResult)
+	if method, ok := ii.methodMap[name]; ok {
+		res, err := method.MethodFunc(ctx, args, ii.svc)
+		result.SetResult(res)
+		result.SetError(err)
+		return result
+	}
+	result.SetError(fmt.Errorf("no match method for %s", name))
+
+	return result
+}
+
+func newInfoInvoker(url *common.URL, info *ServiceInfo, svc common.RPCService) protocol.Invoker {
+	invoker := &infoInvoker{
+		base: protocol.NewBaseInvoker(url),
+		info: info,
+		svc:  svc,
+	}
+	invoker.init()
+	return invoker
+}
+
+// Register assemble invoker chains like ProviderConfig.Load, init a service per call
+func (s *Server) Register(handler interface{}, info *ServiceInfo, opts ...ServiceOption) error {
+	if s.cfg == nil {
+		return errors.New("Server has not been initialized, please use NewServer() to create Server")
+	}
+	var svcOpts []ServiceOption
+	appCfg := s.cfg.Application
+	proCfg := s.cfg.Provider
+	prosCfg := s.cfg.Protocols
+	regsCfg := s.cfg.Registries
+	// todo(DMwangnima): record the registered service
+	newSvcOpts := defaultServiceOptions()
+	if appCfg != nil {
+		svcOpts = append(svcOpts,
+			SetApplication(s.cfg.Application),
+		)
+	}
+	if proCfg != nil {
+		svcOpts = append(svcOpts,
+			SetProvider(proCfg),
+		)
+	}
+	if prosCfg != nil {
+		svcOpts = append(svcOpts,
+			SetProtocols(prosCfg),
+		)
+	}
+	if regsCfg != nil {
+		svcOpts = append(svcOpts,
+			SetRegistries(regsCfg),
+		)
+	}
+
+	// options passed by users have higher priority
+	svcOpts = append(svcOpts, opts...)
+	if err := newSvcOpts.init(svcOpts...); err != nil {
+		return err
+	}
+	newSvcOpts.Implement(handler)
+	if err := newSvcOpts.ExportWithInfo(info); err != nil {
+		return err
+	}
+
+	return nil
+}
+
+func (s *Server) Serve() error {
+	metadata.ExportMetadataService()
+	registry_exposed.RegisterServiceInstance(s.cfg.Application.Name, s.cfg.Application.Tag, s.cfg.Application.MetadataType)
+	select {}
+	return nil
+}
+
+type MethodInfo struct {
+	Name           string
+	Type           string
+	ReqInitFunc    func() interface{}
+	StreamInitFunc func(baseStream interface{}) interface{}
+	MethodFunc     func(ctx context.Context, args []interface{}, handler interface{}) (interface{}, error)
+	Meta           map[string]interface{}
+}
+
+func NewServer(opts ...ServerOption) (*Server, error) {
+	newSrvOpts := defaultServerOptions()
+	if err := newSrvOpts.init(opts...); err != nil {
+		return nil, err
+	}
+	return &Server{
+		cfg: newSrvOpts,
+	}, nil
+}