From c1caae019a3130c25a16db2bd9e05c9bcb98ec35 Mon Sep 17 00:00:00 2001 From: YangruiEmma Date: Mon, 6 Jan 2025 18:53:06 +0800 Subject: [PATCH 01/35] fix: fix the issue which the server-side did not report failure after panic occurred during request processing (#1655) --- pkg/remote/trans/default_server_handler.go | 31 ++++++----- .../trans/default_server_handler_test.go | 54 +++++++++++++++++++ .../trans/netpoll/server_handler_test.go | 4 +- 3 files changed, 74 insertions(+), 15 deletions(-) diff --git a/pkg/remote/trans/default_server_handler.go b/pkg/remote/trans/default_server_handler.go index 6643c035ec..3d91c206d4 100644 --- a/pkg/remote/trans/default_server_handler.go +++ b/pkg/remote/trans/default_server_handler.go @@ -88,11 +88,19 @@ func (t *svrTransHandler) Write(ctx context.Context, conn net.Conn, sendMsg remo // Read implements the remote.ServerTransHandler interface. func (t *svrTransHandler) Read(ctx context.Context, conn net.Conn, recvMsg remote.Message) (nctx context.Context, err error) { var bufReader remote.ByteBuffer + ri := recvMsg.RPCInfo() defer func() { + if r := recover(); r != nil { + stack := string(debug.Stack()) + panicErr := kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in Read] %s", r), stack) + rpcinfo.AsMutableRPCStats(ri.Stats()).SetPanicked(panicErr) + err = remote.NewTransError(remote.ProtocolError, panicErr) + nctx = ctx + } t.ext.ReleaseBuffer(bufReader, err) - rpcinfo.Record(ctx, recvMsg.RPCInfo(), stats.ReadFinish, err) + rpcinfo.Record(ctx, ri, stats.ReadFinish, err) }() - rpcinfo.Record(ctx, recvMsg.RPCInfo(), stats.ReadStart, nil) + rpcinfo.Record(ctx, ri, stats.ReadStart, nil) bufReader = t.ext.NewReadByteBuffer(ctx, conn, recvMsg) if codec, ok := t.codec.(remote.MetaDecoder); ok { @@ -133,9 +141,8 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) var sendMsg remote.Message closeConnOutsideIfErr := true defer func() { - panicErr := recover() - var wrapErr error - if panicErr != nil { + var panicErr error + if r := recover(); r != nil { stack := string(debug.Stack()) if conn != nil { ri := rpcinfo.GetRPCInfo(ctx) @@ -144,10 +151,9 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) } else { klog.CtxErrorf(ctx, "KITEX: panic happened, error=%v\nstack=%s", panicErr, stack) } - if err != nil { - wrapErr = kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in OnRead] %s, last error=%s", panicErr, err.Error()), stack) - } else { - wrapErr = kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in OnRead] %s", panicErr), stack) + panicErr = kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in OnRead] %s", panicErr), stack) + if err == nil { + err = panicErr } } t.finishTracer(ctx, ri, err, panicErr) @@ -158,10 +164,9 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) if rpcinfo.PoolEnabled() { t.opt.InitOrResetRPCInfoFunc(ri, conn.RemoteAddr()) } - if wrapErr != nil { - err = wrapErr - } if err != nil && !closeConnOutsideIfErr { + // when error is not nil, outside will close conn, + // set err to nil to indicate that this kind of error does not require closing the connection err = nil } }() @@ -186,7 +191,7 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) // reply processing var methodInfo serviceinfo.MethodInfo if methodInfo, err = GetMethodInfo(ri, svcInfo); err != nil { - // it won't be err, because the method has been checked in decode, err check here just do defensive inspection + // it won't be error, because the method has been checked in decode, err check here just do defensive inspection t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, true) // for proxy case, need read actual remoteAddr, error print must exec after writeErrorReplyIfNeeded, // t.OnError(ctx, err, conn) will be executed at outer function when transServer close the conn diff --git a/pkg/remote/trans/default_server_handler_test.go b/pkg/remote/trans/default_server_handler_test.go index af750ddae3..31d43da6b9 100644 --- a/pkg/remote/trans/default_server_handler_test.go +++ b/pkg/remote/trans/default_server_handler_test.go @@ -20,6 +20,7 @@ import ( "context" "errors" "net" + "strings" "testing" "github.com/golang/mock/gomock" @@ -211,6 +212,59 @@ func TestSvrTransHandlerReadErr(t *testing.T) { test.Assert(t, errors.Is(err, mockErr)) } +func TestSvrTransHandlerReadPanic(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + mockTracer := stats.NewMockTracer(ctrl) + mockTracer.EXPECT().Start(gomock.Any()).DoAndReturn(func(ctx context.Context) context.Context { return ctx }).AnyTimes() + mockTracer.EXPECT().Finish(gomock.Any()).DoAndReturn(func(ctx context.Context) { + err := rpcinfo.GetRPCInfo(ctx).Stats().Error() + test.Assert(t, err != nil) + }).AnyTimes() + + buf := remote.NewReaderWriterBuffer(1024) + ext := &MockExtension{ + NewWriteByteBufferFunc: func(ctx context.Context, conn net.Conn, msg remote.Message) remote.ByteBuffer { + return buf + }, + NewReadByteBufferFunc: func(ctx context.Context, conn net.Conn, msg remote.Message) remote.ByteBuffer { + return buf + }, + } + + tracerCtl := &rpcinfo.TraceController{} + tracerCtl.Append(mockTracer) + opt := &remote.ServerOption{ + Codec: &MockCodec{ + EncodeFunc: func(ctx context.Context, msg remote.Message, out remote.ByteBuffer) error { + return nil + }, + DecodeFunc: func(ctx context.Context, msg remote.Message, in remote.ByteBuffer) error { + panic("mock") + }, + }, + SvcSearcher: svcSearcher, + TargetSvcInfo: svcInfo, + TracerCtl: tracerCtl, + InitOrResetRPCInfoFunc: func(ri rpcinfo.RPCInfo, addr net.Addr) rpcinfo.RPCInfo { + rpcinfo.AsMutableEndpointInfo(ri.From()).SetAddress(addr) + return ri + }, + } + ri := rpcinfo.NewRPCInfo(rpcinfo.EmptyEndpointInfo(), rpcinfo.FromBasicInfo(&rpcinfo.EndpointBasicInfo{}), + rpcinfo.NewInvocation("", ""), nil, rpcinfo.NewRPCStats()) + ctx := rpcinfo.NewCtxWithRPCInfo(context.Background(), ri) + + svrHandler, err := NewDefaultSvrTransHandler(opt, ext) + test.Assert(t, err == nil) + pl := remote.NewTransPipeline(svrHandler) + svrHandler.SetPipeline(pl) + err = svrHandler.OnRead(ctx, &mocks.Conn{}) + test.Assert(t, err != nil) + test.Assert(t, strings.Contains(err.Error(), "panic")) +} + func TestSvrTransHandlerOnReadHeartbeat(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() diff --git a/pkg/remote/trans/netpoll/server_handler_test.go b/pkg/remote/trans/netpoll/server_handler_test.go index fed95dd5a5..2693886685 100644 --- a/pkg/remote/trans/netpoll/server_handler_test.go +++ b/pkg/remote/trans/netpoll/server_handler_test.go @@ -172,8 +172,8 @@ func TestInvokeErr(t *testing.T) { test.Assert(t, isInvoked) } -// TestPanicAfterRead test server_handler not panic after read -func TestPanicAfterRead(t *testing.T) { +// TestPipelineNilPanic test server_handler that TransPipeline is nil +func TestPipelineNilPanic(t *testing.T) { // 1. prepare mock data var isWriteBufFlushed bool var isReaderBufReleased bool From 48fcf423b44da4f289d9756850f19a9b69e8e558 Mon Sep 17 00:00:00 2001 From: Yi Duan Date: Tue, 7 Jan 2025 19:19:50 +0800 Subject: [PATCH 02/35] feat(endpoint): `MiddlewareBuilder` can return nil (#1661) --- client/client.go | 4 +++- server/server.go | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/client/client.go b/client/client.go index eeb7ec8a20..2c81fe8a14 100644 --- a/client/client.go +++ b/client/client.go @@ -310,7 +310,9 @@ func (kc *kClient) initStreamMiddlewares(ctx context.Context) { func richMWsWithBuilder(ctx context.Context, mwBs []endpoint.MiddlewareBuilder) (mws []endpoint.Middleware) { for i := range mwBs { - mws = append(mws, mwBs[i](ctx)) + if mw := mwBs[i](ctx); mw != nil { + mws = append(mws, mw) + } } return } diff --git a/server/server.go b/server/server.go index 14932b71a3..025e493173 100644 --- a/server/server.go +++ b/server/server.go @@ -154,7 +154,9 @@ func (s *server) buildMiddlewares(ctx context.Context) []endpoint.Middleware { } // register server middlewares for i := range s.opt.MWBs { - mws = append(mws, s.opt.MWBs[i](ctx)) + if mw := s.opt.MWBs[i](ctx); mw != nil { + mws = append(mws, mw) + } } // register services middlewares if mw := s.buildServiceMiddleware(); mw != nil { From 962977b8dc925b7ce5a5c5b80a3b8a0c82d7b618 Mon Sep 17 00:00:00 2001 From: Xuran <37136584+Duslia@users.noreply.github.com> Date: Thu, 9 Jan 2025 18:34:45 +0800 Subject: [PATCH 03/35] fix(serviceinline): fix the issue of missing CtxEventBusKey and CtxEventQueueKey during server initialization in the service inline scenario (#1659) --- client/service_inline.go | 2 ++ server/service_inline.go | 10 +++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/client/service_inline.go b/client/service_inline.go index c807d749e3..2c9e0e5556 100644 --- a/client/service_inline.go +++ b/client/service_inline.go @@ -14,6 +14,8 @@ * limitations under the License. */ +// Service inline is a service deployment form of ByteDance's internal applications. +// Different Kitex services are merged together during the compilation period through the tool chain, and this capability is not yet opensource. package client import ( diff --git a/server/service_inline.go b/server/service_inline.go index ca512be2f0..1a780ae820 100644 --- a/server/service_inline.go +++ b/server/service_inline.go @@ -14,6 +14,8 @@ * limitations under the License. */ +// Service inline is a service deployment form of ByteDance's internal applications. +// Different Kitex services are merged together during the compilation period through the tool chain, and this capability is not yet opensource. package server import ( @@ -128,8 +130,14 @@ func (s *server) BuildServiceInlineInvokeChain() endpoint.Endpoint { return } } + + // In the internal service inline scenario, if the merged service does not detect the RPC Ingress Mesh being enabled + // (for example, if the main service is an API service, the RPC Ingress Mesh will not be enabled), + // the CtxEventQueueKey will be used to monitor configuration changes for debugging during server initialization. + // If this key is not injected, it will lead to a panic during initialization. + ctx := fillContext(s.opt) mws := []endpoint.Middleware{mw} - smws := s.buildMiddlewares(context.Background()) + smws := s.buildMiddlewares(ctx) mws = append(mws, smws...) return endpoint.Chain(mws...)(innerHandlerEp) } From df5b07148d7b4f0892151371620fed6fdac0dbeb Mon Sep 17 00:00:00 2001 From: xiaok29 <1526783667@qq.com> Date: Fri, 10 Jan 2025 10:37:36 +0800 Subject: [PATCH 04/35] =?UTF-8?q?=F0=9F=8E=88=20perf:=20Delete=20excess=20?= =?UTF-8?q?err=20judgment?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- tool/cmd/kitex/args/args.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index 122aaddafd..a4bba20b9f 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -341,10 +341,6 @@ func (a *Arguments) BuildCmd(out io.Writer) (*exec.Cmd, error) { Stderr: io.MultiWriter(out, os.Stderr), } - if err != nil { - return nil, err - } - if a.IDLType == "thrift" { os.Setenv(EnvPluginMode, thriftgo.PluginName) cmd.Args = append(cmd.Args, "thriftgo") From 91629940ac8e1698ba7bee86e06b1f04ee115300 Mon Sep 17 00:00:00 2001 From: Li2CO3 Date: Mon, 13 Jan 2025 20:17:00 +0800 Subject: [PATCH 05/35] fix(tool): fix kitex tool bugs (#1672) --- tool/cmd/kitex/args/args.go | 17 ++++++++++++++--- tool/cmd/kitex/main.go | 4 +++- .../pluginmode/thriftgo/convertor.go | 14 ++++---------- 3 files changed, 21 insertions(+), 14 deletions(-) diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index a4bba20b9f..5adf00575e 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -136,12 +136,11 @@ func (a *Arguments) buildFlags(version string) *flag.FlagSet { "Generate streaming code with streamx interface", ) - f.BoolVar(&a.GenFrugal, "gen_frugal", false, `Gen frugal codec for those structs with (go.codec="frugal")`) - f.Var(&a.FrugalStruct, "frugal_struct", "Gen frugal codec for given struct") + f.BoolVar(&a.GenFrugal, "gen-frugal", false, `Gen frugal codec for those structs with (go.codec="frugal")`) + f.Var(&a.FrugalStruct, "frugal-struct", "Gen frugal codec for given struct") f.BoolVar(&a.NoRecurse, "no-recurse", false, `Don't generate thrift files recursively, just generate the given file.'`) - a.RecordCmd = os.Args a.Version = version a.ThriftOptions = append(a.ThriftOptions, "naming_style=golint", @@ -179,6 +178,18 @@ func (a *Arguments) ParseArgs(version, curpath string, kitexArgs []string) (err return err } + if a.Record { + a.RecordCmd = os.Args + } + + // format -thrift xxx,xxx to -thrift xx -thrift xx + thriftOptions := make([]string, len(a.ThriftOptions)) + for i := range a.ThriftOptions { + op := a.ThriftOptions[i] + thriftOptions = append(thriftOptions, strings.Split(op, ",")...) + } + a.ThriftOptions = thriftOptions + log.Verbose = a.Verbose for _, e := range a.extends { diff --git a/tool/cmd/kitex/main.go b/tool/cmd/kitex/main.go index 5b09d8655d..a1530837c7 100644 --- a/tool/cmd/kitex/main.go +++ b/tool/cmd/kitex/main.go @@ -104,7 +104,9 @@ func main() { if args.IDLType == "thrift" && !args.LocalThriftgo { err = sdk.InvokeThriftgoBySDK(curpath, cmd) } else { - log.Warn("You are using local thriftgo. Please make sure the version is matched with kitex tool.") + if args.IDLType == "thrift" { + log.Warn("You are using local thriftgo. Please make sure the version is matched with kitex tool.") + } err = kargs.ValidateCMD(cmd.Path, args.IDLType) if err != nil { log.Warn(err) diff --git a/tool/internal_pkg/pluginmode/thriftgo/convertor.go b/tool/internal_pkg/pluginmode/thriftgo/convertor.go index a9ac6b9f27..2137eaaa2d 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/convertor.go +++ b/tool/internal_pkg/pluginmode/thriftgo/convertor.go @@ -295,17 +295,9 @@ func (c *converter) convertTypes(req *plugin.Request) error { c.svc2ast = make(map[*generator.ServiceInfo]*parser.Thrift) - var trees chan *parser.Thrift - if req.Recursive { - trees = req.AST.DepthFirstSearch() - } else { - trees = make(chan *parser.Thrift, 1) - trees <- req.AST - close(trees) - } mainAST := req.AST - for ast := range trees { + for ast := range req.AST.DepthFirstSearch() { ref, pkg, pth := c.Utils.ParseNamespace(ast) // make the current ast as an include to produce correct type references. fake := c.copyTreeWithRef(ast, ref) @@ -411,7 +403,9 @@ func (c *converter) convertTypes(req *plugin.Request) error { c.svc2ast[si] = ast } - c.Services = append(c.Services, all[ast.Filename]...) + if req.Recursive || ast == mainAST { + c.Services = append(c.Services, all[ast.Filename]...) + } } return nil } From c2b6b7609c81b21ad87c3283cd89aadd23f898d7 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Tue, 14 Jan 2025 11:50:12 +0800 Subject: [PATCH 06/35] perf(thrift): optimized skipdecoder for netpoll (#1643) --- go.mod | 2 +- go.sum | 2 + pkg/remote/codec/thrift/codec_apache.go | 216 +++++++++++++++++++ pkg/remote/codec/thrift/codec_apache_test.go | 120 +++++++++++ pkg/remote/trans/netpoll/bytebuf.go | 10 + 5 files changed, 349 insertions(+), 1 deletion(-) diff --git a/go.mod b/go.mod index 69ac519ae3..a14106df82 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,7 @@ require ( github.com/cloudwego/dynamicgo v0.4.7-0.20241220085612-55704ea4ca8f github.com/cloudwego/fastpb v0.0.5 github.com/cloudwego/frugal v0.2.3 - github.com/cloudwego/gopkg v0.1.3 + github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 github.com/cloudwego/localsession v0.1.1 github.com/cloudwego/netpoll v0.6.5 github.com/cloudwego/runtimex v0.1.0 diff --git a/go.sum b/go.sum index 8f59dbebc6..a10a556e2e 100644 --- a/go.sum +++ b/go.sum @@ -24,6 +24,8 @@ github.com/cloudwego/frugal v0.2.3 h1:t1hhhAi8lXcx7Ncs4PR1pSZ90vlDU1cy5K2btDMFpo github.com/cloudwego/frugal v0.2.3/go.mod h1:nC1U47gswLRiaxv6dybrhZvsDGCfQP9RGiiWC73CnoI= github.com/cloudwego/gopkg v0.1.3 h1:y9VA5Zn5yqd1+QBV9aB0Zxy56JlAS7x4ZUoED/vJdxA= github.com/cloudwego/gopkg v0.1.3/go.mod h1:FQuXsRWRsSqJLsMVd5SYzp8/Z1y5gXKnVvRrWUOsCMI= +github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 h1:RmKES3lXoM62rUVEEy0teTsPYNBtFpqSN1lYBw4v82g= +github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7/go.mod h1:FQuXsRWRsSqJLsMVd5SYzp8/Z1y5gXKnVvRrWUOsCMI= github.com/cloudwego/iasm v0.2.0 h1:1KNIy1I1H9hNNFEEH3DVnI4UujN+1zjpuk6gwHLTssg= github.com/cloudwego/iasm v0.2.0/go.mod h1:8rXZaNYT2n95jn+zTI1sDr+IgcD2GVs0nlbbQPiEFhY= github.com/cloudwego/localsession v0.1.1 h1:tbK7laDVrYfFDXoBXo4uCGMAxU4qmz2dDm8d4BGBnDo= diff --git a/pkg/remote/codec/thrift/codec_apache.go b/pkg/remote/codec/thrift/codec_apache.go index e2f3026d5e..7b632bcdf2 100644 --- a/pkg/remote/codec/thrift/codec_apache.go +++ b/pkg/remote/codec/thrift/codec_apache.go @@ -18,10 +18,14 @@ package thrift import ( "context" + "encoding/binary" + "fmt" + "sync" "github.com/cloudwego/gopkg/bufiox" "github.com/cloudwego/gopkg/protocol/thrift" "github.com/cloudwego/gopkg/protocol/thrift/apache" + "github.com/cloudwego/netpoll" "github.com/cloudwego/kitex/pkg/remote" ) @@ -33,7 +37,28 @@ func apacheCodecAvailable(data interface{}) bool { // skipThriftStruct is used to read a struct bytes from trans when dataLen = 0, // so that we can use frugalUnmarshal or fastUnmarshal instead of apacheUnmarshal func skipThriftStruct(trans bufiox.Reader) ([]byte, error) { + p, ok := trans.(interface { + NetpollReader() netpoll.Reader + }) + if !ok { + return skipThriftStructSlow(trans) + } + r := p.NetpollReader() + if r == nil { + return skipThriftStructSlow(trans) + } + x := newNetpollSkipDecoder(r) + n, err := x.SkipStruct() + x.Release() + if err != nil { + return nil, err + } + return trans.Next(n) +} + +func skipThriftStructSlow(trans bufiox.Reader) ([]byte, error) { sd := thrift.NewSkipDecoder(trans) + defer sd.Release() buf, err := sd.Next(thrift.STRUCT) if err != nil { return nil, remote.NewTransError(remote.ProtocolError, err).AppendMessage("caught in skipThriftStruct") @@ -74,3 +99,194 @@ func apacheMarshalData(data interface{}) (buf []byte, err error) { _ = bw.Flush() return buf, nil } + +type netpollSkipDecoder struct { + n int + b []byte + r netpoll.Reader +} + +var skipdecoderPool = sync.Pool{ + New: func() any { return &netpollSkipDecoder{} }, +} + +func newNetpollSkipDecoder(r netpoll.Reader) *netpollSkipDecoder { + p := skipdecoderPool.Get().(*netpollSkipDecoder) + p.Reset(r) + return p +} + +func (p *netpollSkipDecoder) Reset(r netpoll.Reader) { + *p = netpollSkipDecoder{r: r} +} + +func (p *netpollSkipDecoder) Release() { + skipdecoderPool.Put(p) +} + +// skipn is small enough that can be inlined. +// +// XXX: as a trade off, +// caller have to use lastbyte or lastbytes after calling skipn without err +func (p *netpollSkipDecoder) skipn(n int) error { + if p.n+n <= len(p.b) { + p.n += n + return nil + } + return p.skipnSlow(n) +} + +func (p *netpollSkipDecoder) lastbyte() byte { + return p.b[p.n-1] +} + +func (p *netpollSkipDecoder) lastbytes(n int) (b []byte) { + return p.b[p.n-n:] +} + +func (p *netpollSkipDecoder) skipnSlow(n int) error { + sz := p.n + n + + // trigger underlying conn to read more + if l := p.r.Len(); l > sz { + // read as much as possible, luckly, we will have a full buffer + // then we no need to call p.Peek many times + sz = l + } + b, err := p.r.Peek(sz) + if err != nil { + return err + } + p.b = b + p.n += n + return nil +} + +func (p *netpollSkipDecoder) SkipStruct() (int, error) { + const defaultRecursionDepth = 64 + if err := p.skipType(thrift.STRUCT, defaultRecursionDepth); err != nil { + return 0, err + } + return p.n, nil +} + +var ( + errDepthLimitExceeded = thrift.NewProtocolException( + thrift.DEPTH_LIMIT, "depth limit exceeded") + + errDataLength = thrift.NewProtocolException( + thrift.INVALID_DATA, "invalid data length") +) + +var typeToSize = [256]int8{ + thrift.BOOL: 1, + thrift.BYTE: 1, + thrift.DOUBLE: 8, + thrift.I16: 2, + thrift.I32: 4, + thrift.I64: 8, +} + +func (p *netpollSkipDecoder) skipType(t thrift.TType, maxdepth int) error { + if maxdepth == 0 { + return errDepthLimitExceeded + } + if sz := typeToSize[t]; sz > 0 { + return p.skipn(int(sz)) + } + switch t { + case thrift.STRING: + if err := p.skipn(4); err != nil { + return err + } + sz := int(int32(binary.BigEndian.Uint32(p.lastbytes(4)))) + if sz < 0 { + return errDataLength + } + if err := p.skipn(sz); err != nil { + return err + } + case thrift.STRUCT: + for { + if err := p.skipn(1); err != nil { // TType + return err + } + tp := thrift.TType(p.lastbyte()) + if tp == thrift.STOP { + break + } + if sz := typeToSize[tp]; sz > 0 { + // fastpath + // Field ID + Value + if err := p.skipn(2 + int(sz)); err != nil { + return err + } + continue + } + + // Field ID + if err := p.skipn(2); err != nil { + return err + } + // Field Value + if err := p.skipType(tp, maxdepth-1); err != nil { + return err + } + } + case thrift.MAP: + // 1 byte key TType, 1 byte value TType, 4 bytes Len + if err := p.skipn(6); err != nil { + return err + } + b := p.lastbytes(6) + kt, vt, sz := thrift.TType(b[0]), thrift.TType(b[1]), int32(binary.BigEndian.Uint32(b[2:])) + if sz < 0 { + return errDataLength + } + ksz, vsz := int(typeToSize[kt]), int(typeToSize[vt]) + if ksz > 0 && vsz > 0 { + return p.skipn(int(sz) * (ksz + vsz)) + } + var err error + for i := int32(0); i < sz; i++ { + if ksz > 0 { + err = p.skipn(ksz) + } else { + err = p.skipType(kt, maxdepth-1) + } + if err != nil { + return err + } + if vsz > 0 { + err = p.skipn(vsz) + } else { + err = p.skipType(vt, maxdepth-1) + } + if err != nil { + return err + } + } + case thrift.SET, thrift.LIST: + // 1 byte value type, 4 bytes Len + if err := p.skipn(5); err != nil { + return err + } + b := p.lastbytes(5) + vt, sz := thrift.TType(b[0]), int32(binary.BigEndian.Uint32(b[1:])) + if sz < 0 { + return errDataLength + } + if vsz := typeToSize[vt]; vsz > 0 { + return p.skipn(int(sz) * int(vsz)) + } + for i := int32(0); i < sz; i++ { + if err := p.skipType(vt, maxdepth-1); err != nil { + return err + } + } + default: + return thrift.NewProtocolException( + thrift.INVALID_DATA, fmt.Sprintf("unknown data type %d", t)) + } + return nil +} diff --git a/pkg/remote/codec/thrift/codec_apache_test.go b/pkg/remote/codec/thrift/codec_apache_test.go index 2a951ead4d..b35bceca56 100644 --- a/pkg/remote/codec/thrift/codec_apache_test.go +++ b/pkg/remote/codec/thrift/codec_apache_test.go @@ -26,9 +26,13 @@ import ( "github.com/cloudwego/gopkg/bufiox" "github.com/cloudwego/gopkg/protocol/thrift" "github.com/cloudwego/gopkg/protocol/thrift/apache" + "github.com/cloudwego/netpoll" + "github.com/golang/mock/gomock" + mnetpoll "github.com/cloudwego/kitex/internal/mocks/netpoll" "github.com/cloudwego/kitex/internal/test" "github.com/cloudwego/kitex/pkg/remote" + tnetpoll "github.com/cloudwego/kitex/pkg/remote/trans/netpoll" ) func TestGetSkippedStructBuffer(t *testing.T) { @@ -107,3 +111,119 @@ func TestApacheMarshalUnmarshal(t *testing.T) { test.Assert(t, err == nil, err) test.Assert(t, p.TypeID() == ex.TypeID() && p.Msg() == ex.Msg()) } + +func genTestSkipDecoderBytes() []byte { + x := thrift.BinaryProtocol{} + + b := []byte{} + + // simple struct for reuse + structb := x.AppendFieldBegin([]byte{}, thrift.I64, 10001) + structb = x.AppendI64(structb, 100010) + structb = x.AppendFieldStop(structb) + + // scalar types + b = x.AppendFieldBegin(b, thrift.BOOL, 1) + b = x.AppendBool(b, true) + b = x.AppendFieldBegin(b, thrift.BYTE, 2) + b = x.AppendByte(b, 2) + b = x.AppendFieldBegin(b, thrift.DOUBLE, 3) + b = x.AppendDouble(b, 3) + b = x.AppendFieldBegin(b, thrift.I16, 4) + b = x.AppendI16(b, 4) + b = x.AppendFieldBegin(b, thrift.I32, 5) + b = x.AppendI32(b, 5) + b = x.AppendFieldBegin(b, thrift.I64, 6) + b = x.AppendI64(b, 6) + + const elements = 10 + + // Struct + b = x.AppendFieldBegin(b, thrift.STRUCT, 7) + b = append(b, structb...) + + // List with scalar type + b = x.AppendFieldBegin(b, thrift.LIST, 101) + b = x.AppendListBegin(b, thrift.I64, elements) + for i := 0; i < elements; i++ { + b = x.AppendI64(b, 1011) + } + + // List with nested type + b = x.AppendFieldBegin(b, thrift.LIST, 102) + b = x.AppendListBegin(b, thrift.STRUCT, elements) + for i := 0; i < elements; i++ { + b = append(b, structb...) + } + + // Map with scalar type + b = x.AppendFieldBegin(b, thrift.MAP, 201) + b = x.AppendMapBegin(b, thrift.I64, thrift.DOUBLE, elements) + for i := 0; i < elements; i++ { + b = x.AppendI64(b, 2011) + b = x.AppendDouble(b, 2012.2) + } + + // Map with string key + b = x.AppendFieldBegin(b, thrift.MAP, 202) + b = x.AppendMapBegin(b, thrift.STRING, thrift.I64, elements) + for i := 0; i < elements; i++ { + b = x.AppendString(b, "hello-202") + b = x.AppendI64(b, 2022) + } + + // Map with nested value + b = x.AppendFieldBegin(b, thrift.MAP, 203) + b = x.AppendMapBegin(b, thrift.I64, thrift.STRUCT, elements) + for i := 0; i < elements; i++ { + b = x.AppendI64(b, 2031) + b = append(b, structb...) + } + + return x.AppendFieldStop(b) +} + +func TestNetpollSkipDecoder(t *testing.T) { + ctrl := gomock.NewController(t) + defer ctrl.Finish() + + b := genTestSkipDecoderBytes() + + r := mnetpoll.NewMockReader(ctrl) + r.EXPECT().Len().Times(1).Return(len(b)) + r.EXPECT().Peek(len(b)).Times(1).Return(b, nil) + r.EXPECT().Next(len(b)).Times(1).Return(b, nil) + + returnb, err := skipThriftStruct(tnetpoll.NewReaderByteBuffer(r)) + test.Assert(t, err == nil, err) + test.Assert(t, bytes.Equal(returnb, b)) +} + +type mockNetpollReader struct { + netpoll.Reader + + b []byte +} + +func (r *mockNetpollReader) Len() int { return len(r.b) } +func (r *mockNetpollReader) Peek(n int) ([]byte, error) { return r.b[:n], nil } + +func BenchmarkNetpollSkipDecoder(tb *testing.B) { + tb.ResetTimer() + r := &mockNetpollReader{b: genTestSkipDecoderBytes()} + p := newNetpollSkipDecoder(r) + for i := 0; i < tb.N; i++ { + p.Reset(r) + p.SkipStruct() + } +} + +func BenchmarkGopkgByteSkipDecoder(tb *testing.B) { + b := genTestSkipDecoderBytes() + tb.ResetTimer() + p := thrift.NewBytesSkipDecoder(b) + for i := 0; i < tb.N; i++ { + p.Reset(b) + p.Next(thrift.STRUCT) + } +} diff --git a/pkg/remote/trans/netpoll/bytebuf.go b/pkg/remote/trans/netpoll/bytebuf.go index a1ceb5f475..214bacf944 100644 --- a/pkg/remote/trans/netpoll/bytebuf.go +++ b/pkg/remote/trans/netpoll/bytebuf.go @@ -70,6 +70,16 @@ type netpollByteBuffer struct { var _ remote.ByteBuffer = &netpollByteBuffer{} +// NetpollReader returns the underlying netpoll reader, nil if not available +// +// This method only used by skip decoder for performance concern. +func (b *netpollByteBuffer) NetpollReader() netpoll.Reader { + if b.status&remote.BitReadable == 0 { + return nil + } + return b.reader +} + // Next reads n bytes sequentially, returns the original address. func (b *netpollByteBuffer) Next(n int) (p []byte, err error) { if b.status&remote.BitReadable == 0 { From 99c521697c62abda7d688fd2a3682e2dd24ae677 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Tue, 14 Jan 2025 11:51:59 +0800 Subject: [PATCH 07/35] test: fix streaming unit tests (#1671) --- pkg/streamx/streamx_user_test.go | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/pkg/streamx/streamx_user_test.go b/pkg/streamx/streamx_user_test.go index 900dfb6119..8f316e4104 100644 --- a/pkg/streamx/streamx_user_test.go +++ b/pkg/streamx/streamx_user_test.go @@ -803,13 +803,12 @@ func TestStreamingTracing(t *testing.T) { test.Assert(t, err == nil, err) defer svr.Stop() tracer := &mockTracer{} - var finished sync.WaitGroup + finishedChan := make(chan struct{}, 1) tracer.start = func(ctx context.Context) context.Context { - finished.Add(1) return ctx } tracer.finish = func(ctx context.Context) { - finished.Done() + finishedChan <- struct{}{} } cli, err := NewClient( "kitex.echo.service", @@ -830,7 +829,7 @@ func TestStreamingTracing(t *testing.T) { test.Assert(t, ri.Stats().Error() == nil, ri.Stats().Error()) _, err = stream.Recv(ctx) test.Assert(t, err != nil, err) - finished.Wait() + <-finishedChan // stream has been closed, Tracer.Finish should be called // err set in rpcinfo.Stats() should be the same as the err returned to users test.Assert(t, ri.Stats().Error() == err, ri.Stats().Error()) @@ -848,7 +847,7 @@ func TestStreamingTracing(t *testing.T) { test.Assert(t, ri.Stats().Error() == nil, ri.Stats().Error()) _, err = stream.CloseAndRecv(ctx) test.Assert(t, err != nil, err) - finished.Wait() + <-finishedChan // stream has been closed, Tracer.Finish should be called // err set in rpcinfo.Stats() should be the same as the err returned to users test.Assert(t, ri.Stats().Error() == err, ri.Stats().Error()) @@ -868,7 +867,7 @@ func TestStreamingTracing(t *testing.T) { test.Assert(t, err == nil, err) _, err = stream.Recv(ctx) test.Assert(t, err != nil, err) - finished.Wait() + <-finishedChan // stream has been closed, Tracer.Finish should be called // err set in rpcinfo.Stats() should be the same as the err returned to users test.Assert(t, ri.Stats().Error() == err, ri.Stats().Error()) From cdfa0841ccc91a7ac044bc31a1d6ad0fe4d29205 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Wed, 15 Jan 2025 10:55:55 +0800 Subject: [PATCH 08/35] fix(tool): remove redundant dependency for streamx generation code (#1675) --- tool/internal_pkg/generator/generator.go | 11 ++++++++--- tool/internal_pkg/tpl/client.go | 7 +------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/tool/internal_pkg/generator/generator.go b/tool/internal_pkg/generator/generator.go index 924a112f62..885b1f8945 100644 --- a/tool/internal_pkg/generator/generator.go +++ b/tool/internal_pkg/generator/generator.go @@ -546,9 +546,14 @@ func (g *generator) setImports(name string, pkg *PackageInfo) { switch name { case ClientFileName: pkg.AddImports("client") - if !g.StreamX && pkg.HasStreaming { - pkg.AddImport("streaming", "github.com/cloudwego/kitex/pkg/streaming") - pkg.AddImport("transport", "github.com/cloudwego/kitex/transport") + if pkg.HasStreaming { + if !g.StreamX { + pkg.AddImport("streaming", "github.com/cloudwego/kitex/pkg/streaming") + pkg.AddImport("transport", "github.com/cloudwego/kitex/transport") + } else { + pkg.AddImports("github.com/cloudwego/kitex/client/streamxclient/streamxcallopt") + pkg.AddImports("github.com/cloudwego/kitex/pkg/streamx") + } } if len(pkg.AllMethods()) > 0 { if needCallOpt(pkg) { diff --git a/tool/internal_pkg/tpl/client.go b/tool/internal_pkg/tpl/client.go index b362b26daa..1d901563e2 100644 --- a/tool/internal_pkg/tpl/client.go +++ b/tool/internal_pkg/tpl/client.go @@ -28,14 +28,9 @@ import ( {{- end}} {{- end}} {{- end}} - {{- if .HasStreaming}} - {{- if not .StreamX}} + {{- if and .HasStreaming (not .StreamX) }} "github.com/cloudwego/kitex/client/streamclient" "github.com/cloudwego/kitex/client/callopt/streamcall" - {{- else}} - "github.com/cloudwego/kitex/client/streamxclient/streamxcallopt" - "github.com/cloudwego/kitex/pkg/streamx" - {{- end}}{{- /* if not .StreamX end */}} {{- end}} ) // Client is designed to provide IDL-compatible methods with call-option parameter for kitex framework. From 3e3db535f46fa9dfba9c5b3e3427cedea88a3cd3 Mon Sep 17 00:00:00 2001 From: Yi Duan Date: Wed, 15 Jan 2025 14:37:53 +0800 Subject: [PATCH 09/35] chore: update deps and adjust dynamicgo's default option (#1667) --- go.mod | 11 +++++------ go.sum | 25 ++++++++++--------------- pkg/generic/option.go | 16 +++++++++------- 3 files changed, 24 insertions(+), 28 deletions(-) diff --git a/go.mod b/go.mod index a14106df82..816015e568 100644 --- a/go.mod +++ b/go.mod @@ -4,15 +4,15 @@ go 1.18 require ( github.com/bytedance/gopkg v0.1.1 - github.com/bytedance/sonic v1.12.5 + github.com/bytedance/sonic v1.12.7 github.com/cloudwego/configmanager v0.2.2 - github.com/cloudwego/dynamicgo v0.4.7-0.20241220085612-55704ea4ca8f + github.com/cloudwego/dynamicgo v0.5.0 github.com/cloudwego/fastpb v0.0.5 github.com/cloudwego/frugal v0.2.3 github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 - github.com/cloudwego/localsession v0.1.1 + github.com/cloudwego/localsession v0.1.2 github.com/cloudwego/netpoll v0.6.5 - github.com/cloudwego/runtimex v0.1.0 + github.com/cloudwego/runtimex v0.1.1 github.com/cloudwego/thriftgo v0.3.18 github.com/golang/mock v1.6.0 github.com/google/pprof v0.0.0-20240727154555-813a5fbdbec8 @@ -29,7 +29,7 @@ require ( ) require ( - github.com/bytedance/sonic/loader v0.2.0 // indirect + github.com/bytedance/sonic/loader v0.2.2 // indirect github.com/cloudwego/base64x v0.1.4 // indirect github.com/cloudwego/iasm v0.2.0 // indirect github.com/davecgh/go-spew v1.1.2-0.20180830191138-d8f796af33cc // indirect @@ -39,7 +39,6 @@ require ( github.com/iancoleman/strcase v0.2.0 // indirect github.com/klauspost/cpuid/v2 v2.2.4 // indirect github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421 // indirect - github.com/modern-go/gls v0.0.0-20220109145502-612d0167dce5 // indirect github.com/modern-go/reflect2 v1.0.2 // indirect github.com/pmezard/go-difflib v1.0.0 // indirect github.com/stretchr/testify v1.9.0 // indirect diff --git a/go.sum b/go.sum index a10a556e2e..1285dd6672 100644 --- a/go.sum +++ b/go.sum @@ -1,39 +1,36 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= -github.com/bytedance/gopkg v0.0.0-20230728082804-614d0af6619b/go.mod h1:FtQG3YbQG9L/91pbKSw787yBQPutC+457AvDW77fgUQ= github.com/bytedance/gopkg v0.1.0/go.mod h1:FtQG3YbQG9L/91pbKSw787yBQPutC+457AvDW77fgUQ= github.com/bytedance/gopkg v0.1.1 h1:3azzgSkiaw79u24a+w9arfH8OfnQQ4MHUt9lJFREEaE= github.com/bytedance/gopkg v0.1.1/go.mod h1:576VvJ+eJgyCzdjS+c4+77QF3p7ubbtiKARP3TxducM= github.com/bytedance/sonic v1.11.6/go.mod h1:LysEHSvpvDySVdC2f87zGWf6CIKJcAvqab1ZaiQtds4= -github.com/bytedance/sonic v1.12.5 h1:hoZxY8uW+mT+OpkcUWw4k0fDINtOcVavEsGfzwzFU/w= -github.com/bytedance/sonic v1.12.5/go.mod h1:B8Gt/XvtZ3Fqj+iSKMypzymZxw/FVwgIGKzMzT9r/rk= +github.com/bytedance/sonic v1.12.7 h1:CQU8pxOy9HToxhndH0Kx/S1qU/CuS9GnKYrGioDcU1Q= +github.com/bytedance/sonic v1.12.7/go.mod h1:tnbal4mxOMju17EGfknm2XyYcpyCnIROYOEYuemj13I= github.com/bytedance/sonic/loader v0.1.1/go.mod h1:ncP89zfokxS5LZrJxl5z0UJcsk4M4yY2JpfqGeCtNLU= -github.com/bytedance/sonic/loader v0.2.0 h1:zNprn+lsIP06C/IqCHs3gPQIvnvpKbbxyXQP1iU4kWM= -github.com/bytedance/sonic/loader v0.2.0/go.mod h1:ncP89zfokxS5LZrJxl5z0UJcsk4M4yY2JpfqGeCtNLU= +github.com/bytedance/sonic/loader v0.2.2 h1:jxAJuN9fOot/cyz5Q6dUuMJF5OqQ6+5GfA8FjjQ0R4o= +github.com/bytedance/sonic/loader v0.2.2/go.mod h1:N8A3vUdtUebEY2/VQC0MyhYeKUFosQU6FxH2JmUe6VI= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/0Y= github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= github.com/cloudwego/configmanager v0.2.2 h1:sVrJB8gWYTlPV2OS3wcgJSO9F2/9Zbkmcm1Z7jempOU= github.com/cloudwego/configmanager v0.2.2/go.mod h1:ppiyU+5TPLonE8qMVi/pFQk2eL3Q4P7d4hbiNJn6jwI= -github.com/cloudwego/dynamicgo v0.4.7-0.20241220085612-55704ea4ca8f h1:IERXjxDg3Pbatb5z/dR8Qr8XUA1FpDVa73BnwbeQ76U= -github.com/cloudwego/dynamicgo v0.4.7-0.20241220085612-55704ea4ca8f/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= +github.com/cloudwego/dynamicgo v0.5.0 h1:wcmeZIRC6iW/36sId16ktIFvgnyKER9VzhjEsNhw2GU= +github.com/cloudwego/dynamicgo v0.5.0/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= github.com/cloudwego/fastpb v0.0.5 h1:vYnBPsfbAtU5TVz5+f9UTlmSCixG9F9vRwaqE0mZPZU= github.com/cloudwego/fastpb v0.0.5/go.mod h1:Bho7aAKBUtT9RPD2cNVkTdx4yQumfSv3If7wYnm1izk= github.com/cloudwego/frugal v0.2.3 h1:t1hhhAi8lXcx7Ncs4PR1pSZ90vlDU1cy5K2btDMFpoA= github.com/cloudwego/frugal v0.2.3/go.mod h1:nC1U47gswLRiaxv6dybrhZvsDGCfQP9RGiiWC73CnoI= -github.com/cloudwego/gopkg v0.1.3 h1:y9VA5Zn5yqd1+QBV9aB0Zxy56JlAS7x4ZUoED/vJdxA= -github.com/cloudwego/gopkg v0.1.3/go.mod h1:FQuXsRWRsSqJLsMVd5SYzp8/Z1y5gXKnVvRrWUOsCMI= github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 h1:RmKES3lXoM62rUVEEy0teTsPYNBtFpqSN1lYBw4v82g= github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7/go.mod h1:FQuXsRWRsSqJLsMVd5SYzp8/Z1y5gXKnVvRrWUOsCMI= github.com/cloudwego/iasm v0.2.0 h1:1KNIy1I1H9hNNFEEH3DVnI4UujN+1zjpuk6gwHLTssg= github.com/cloudwego/iasm v0.2.0/go.mod h1:8rXZaNYT2n95jn+zTI1sDr+IgcD2GVs0nlbbQPiEFhY= -github.com/cloudwego/localsession v0.1.1 h1:tbK7laDVrYfFDXoBXo4uCGMAxU4qmz2dDm8d4BGBnDo= -github.com/cloudwego/localsession v0.1.1/go.mod h1:kiJxmvAcy4PLgKtEnPS5AXed3xCiXcs7Z+KBHP72Wv8= +github.com/cloudwego/localsession v0.1.2 h1:RBmeLDO5sKr4ujd8iBp5LTMmuVKLdu88jjIneq/fEZ8= +github.com/cloudwego/localsession v0.1.2/go.mod h1:J4uams2YT/2d4t7OI6A7NF7EcG8OlHJsOX2LdPbqoyc= github.com/cloudwego/netpoll v0.6.5 h1:6E/BWhSzQoyLg9Kx/4xiMdIIpovzwBtXvuqSqaTUzDQ= github.com/cloudwego/netpoll v0.6.5/go.mod h1:BtM+GjKTdwKoC8IOzD08/+8eEn2gYoiNLipFca6BVXQ= -github.com/cloudwego/runtimex v0.1.0 h1:HG+WxWoj5/CDChDZ7D99ROwvSMkuNXAqt6hnhTTZDiI= -github.com/cloudwego/runtimex v0.1.0/go.mod h1:23vL/HGV0W8nSCHbe084AgEBdDV4rvXenEUMnUNvUd8= +github.com/cloudwego/runtimex v0.1.1 h1:lheZjFOyKpsq8TsGGfmX9/4O7F0TKpWmB8on83k7GE8= +github.com/cloudwego/runtimex v0.1.1/go.mod h1:23vL/HGV0W8nSCHbe084AgEBdDV4rvXenEUMnUNvUd8= github.com/cloudwego/thriftgo v0.3.18 h1:gnr1vz7G3RbwwCK9AMKHZf63VYGa7ene6WbI9VrBJSw= github.com/cloudwego/thriftgo v0.3.18/go.mod h1:AdLEJJVGW/ZJYvkkYAZf5SaJH+pA3OyC801WSwqcBwI= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= @@ -97,8 +94,6 @@ github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421 h1:ZqeYNhU3OHLH3mGKHDcjJRFFRrJa6eAM5H+CtDdOsPc= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= -github.com/modern-go/gls v0.0.0-20220109145502-612d0167dce5 h1:uiS4zKYKJVj5F3ID+5iylfKPsEQmBEOucSD9Vgmn0i0= -github.com/modern-go/gls v0.0.0-20220109145502-612d0167dce5/go.mod h1:I8AX+yW//L8Hshx6+a1m3bYkwXkpsVjA2795vP4f4oQ= github.com/modern-go/reflect2 v1.0.2 h1:xBagoLtFs94CBntxluKeaWgTMpvLxC4ur3nMaC9Gz0M= github.com/modern-go/reflect2 v1.0.2/go.mod h1:yWuevngMOJpCy52FWWMvUC8ws7m/LJsjYzDa0/r8luk= github.com/nishanths/predeclared v0.0.0-20200524104333-86fad755b4d3/go.mod h1:nt3d53pc1VYcphSCIaYAJtnPYnr3Zyn8fMq2wvPGPso= diff --git a/pkg/generic/option.go b/pkg/generic/option.go index 1ebe319291..7dc72d433d 100644 --- a/pkg/generic/option.go +++ b/pkg/generic/option.go @@ -20,13 +20,15 @@ import "github.com/cloudwego/dynamicgo/conv" var ( DefaultHTTPDynamicGoConvOpts = conv.Options{ - EnableHttpMapping: true, - EnableValueMapping: true, - WriteRequireField: true, - WriteDefaultField: true, - OmitHttpMappingErrors: true, - NoBase64Binary: true, - UseKitexHttpEncoding: true, + EnableHttpMapping: true, + EnableValueMapping: true, + WriteRequireField: true, + WriteDefaultField: true, + OmitHttpMappingErrors: true, + NoBase64Binary: true, + UseKitexHttpEncoding: true, + WriteHttpValueFallback: true, + ReadHttpValueFallback: true, } DefaultJSONDynamicGoConvOpts = conv.Options{ WriteRequireField: true, From 2f49a5f8c8034748188b837d3d82fef09411d030 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Wed, 15 Jan 2025 14:46:34 +0800 Subject: [PATCH 10/35] test(streamx): fix streamx Recv timeout unit test (#1676) --- pkg/streamx/streamx_user_test.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/pkg/streamx/streamx_user_test.go b/pkg/streamx/streamx_user_test.go index 8f316e4104..676c6dfc16 100644 --- a/pkg/streamx/streamx_user_test.go +++ b/pkg/streamx/streamx_user_test.go @@ -619,18 +619,20 @@ func TestStreamingException(t *testing.T) { } }), ) - octx := context.Background() // assert circuitBreaker error + octx := context.Background() atomic.StoreInt32(&circuitBreaker, 1) _, _, err = cli.BidiStream(octx) test.Assert(t, errors.Is(err, circuitBreakerErr), err) atomic.StoreInt32(&circuitBreaker, 0) // assert context deadline error - ctx, cancel := context.WithTimeout(octx, time.Millisecond) - ctx, bs, err := cli.BidiStream(ctx) + octx = context.Background() + ctx, bs, err := cli.BidiStream(octx) test.Assert(t, err == nil, err) + // ctx timeout should be injected before invoking Recv and after creating stream + ctx, cancel := context.WithTimeout(ctx, time.Millisecond) res, err := bs.Recv(ctx) cancel() test.Assert(t, res == nil && err != nil, res, err) From b7777a2d5c1da1f7f2bbca025a747e78d5c0ddb1 Mon Sep 17 00:00:00 2001 From: Marina Sakai <118230951+Marina-Sakai@users.noreply.github.com> Date: Wed, 15 Jan 2025 17:20:15 +0800 Subject: [PATCH 11/35] feat(generic): add an option to specify an IDL service name to be parsed (#1673) --- go.mod | 2 +- go.sum | 4 +- pkg/generic/thrift/parse.go | 46 +++++++--- pkg/generic/thrift/parse_option.go | 11 ++- pkg/generic/thrift/parse_test.go | 15 ++++ pkg/generic/thriftidl_provider.go | 108 +++++++++++++---------- pkg/generic/thriftidl_provider_option.go | 15 +++- pkg/generic/thriftidl_provider_test.go | 108 +++++++++++++++++++++++ 8 files changed, 245 insertions(+), 64 deletions(-) diff --git a/go.mod b/go.mod index 816015e568..cef81f69e7 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/bytedance/gopkg v0.1.1 github.com/bytedance/sonic v1.12.7 github.com/cloudwego/configmanager v0.2.2 - github.com/cloudwego/dynamicgo v0.5.0 + github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71 github.com/cloudwego/fastpb v0.0.5 github.com/cloudwego/frugal v0.2.3 github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 diff --git a/go.sum b/go.sum index 1285dd6672..7c0756ee0b 100644 --- a/go.sum +++ b/go.sum @@ -15,8 +15,8 @@ github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/ github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= github.com/cloudwego/configmanager v0.2.2 h1:sVrJB8gWYTlPV2OS3wcgJSO9F2/9Zbkmcm1Z7jempOU= github.com/cloudwego/configmanager v0.2.2/go.mod h1:ppiyU+5TPLonE8qMVi/pFQk2eL3Q4P7d4hbiNJn6jwI= -github.com/cloudwego/dynamicgo v0.5.0 h1:wcmeZIRC6iW/36sId16ktIFvgnyKER9VzhjEsNhw2GU= -github.com/cloudwego/dynamicgo v0.5.0/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= +github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71 h1:J57+W8YYGJy0MCLk/yzuLehiQmKpOoQng+OBF/5204o= +github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= github.com/cloudwego/fastpb v0.0.5 h1:vYnBPsfbAtU5TVz5+f9UTlmSCixG9F9vRwaqE0mZPZU= github.com/cloudwego/fastpb v0.0.5/go.mod h1:Bho7aAKBUtT9RPD2cNVkTdx4yQumfSv3If7wYnm1izk= github.com/cloudwego/frugal v0.2.3 h1:t1hhhAi8lXcx7Ncs4PR1pSZ90vlDU1cy5K2btDMFpoA= diff --git a/pkg/generic/thrift/parse.go b/pkg/generic/thrift/parse.go index 989ddb7a28..b77e23e366 100644 --- a/pkg/generic/thrift/parse.go +++ b/pkg/generic/thrift/parse.go @@ -77,22 +77,33 @@ func Parse(tree *parser.Thrift, mode ParseMode, opts ...ParseOption) (*descripto Router: descriptor.NewRouter(), } + pOpts := &parseOptions{} + pOpts.apply(opts) + // support one service svcs := tree.Services - switch mode { - case LastServiceOnly: - svcs = svcs[len(svcs)-1:] - sDsc.Name = svcs[len(svcs)-1].Name - case FirstServiceOnly: - svcs = svcs[:1] - sDsc.Name = svcs[0].Name - case CombineServices: - sDsc.Name = "CombinedServices" - sDsc.IsCombinedServices = true - } - pOpts := &parseOptions{} - pOpts.apply(opts) + // if an idl service name is specified, it takes precedence over parse mode + if pOpts.serviceName != "" { + var err error + svcs, err = getTargetService(svcs, pOpts.serviceName) + if err != nil { + return nil, err + } + sDsc.Name = pOpts.serviceName + } else { + switch mode { + case LastServiceOnly: + svcs = svcs[len(svcs)-1:] + sDsc.Name = svcs[len(svcs)-1].Name + case FirstServiceOnly: + svcs = svcs[:1] + sDsc.Name = svcs[0].Name + case CombineServices: + sDsc.Name = "CombinedServices" + sDsc.IsCombinedServices = true + } + } visitedSvcs := make(map[*parser.Service]bool, len(tree.Services)) for _, svc := range svcs { @@ -109,6 +120,15 @@ func Parse(tree *parser.Thrift, mode ParseMode, opts ...ParseOption) (*descripto return sDsc, nil } +func getTargetService(svcs []*parser.Service, serviceName string) ([]*parser.Service, error) { + for _, svc := range svcs { + if svc.Name == serviceName { + return []*parser.Service{svc}, nil + } + } + return nil, fmt.Errorf("the idl service name %s is not in the idl. Please check your idl", serviceName) +} + type pair struct { tree *parser.Thrift data interface{} diff --git a/pkg/generic/thrift/parse_option.go b/pkg/generic/thrift/parse_option.go index f29ff62030..9fffd321a2 100644 --- a/pkg/generic/thrift/parse_option.go +++ b/pkg/generic/thrift/parse_option.go @@ -19,7 +19,8 @@ package thrift import "github.com/cloudwego/kitex/pkg/generic/descriptor" type parseOptions struct { - goTag *descriptor.GoTagOption + goTag *descriptor.GoTagOption + serviceName string } type ParseOption struct { @@ -39,3 +40,11 @@ func WithGoTagDisabled(disable bool) ParseOption { } }} } + +// WithIDLServiceName specifies the target IDL service to be parsed. +// NOTE: with this option, the specified service is prioritized and parse mode will be ignored. +func WithIDLServiceName(serviceName string) ParseOption { + return ParseOption{F: func(opt *parseOptions) { + opt.serviceName = serviceName + }} +} diff --git a/pkg/generic/thrift/parse_test.go b/pkg/generic/thrift/parse_test.go index 30a1088f5e..b9fe9a6192 100644 --- a/pkg/generic/thrift/parse_test.go +++ b/pkg/generic/thrift/parse_test.go @@ -510,3 +510,18 @@ func defaultValueDeepEqual(t *testing.T, defaultValue func(name string) interfac "a": int32(56), })) } + +func TestParseWithIDLServiceName(t *testing.T) { + demo, err := parser.ParseString("demo.thrift", demoIDL) + test.Assert(t, err == nil) + + base, err := parser.ParseString("base.thrift", baseIDL) + test.Assert(t, err == nil) + + demo.Includes[0].Reference = base + + sDsc, err := Parse(demo, LastServiceOnly, WithIDLServiceName("DemoBaseService")) + test.Assert(t, err == nil) + // priority: service name specification > parse mode + test.Assert(t, sDsc.Name == "DemoBaseService") +} diff --git a/pkg/generic/thriftidl_provider.go b/pkg/generic/thriftidl_provider.go index 9a74b8761e..fc89e333ba 100644 --- a/pkg/generic/thriftidl_provider.go +++ b/pkg/generic/thriftidl_provider.go @@ -58,7 +58,7 @@ func NewThriftFileProviderWithOption(path string, opts []ThriftIDLProviderOption } tOpts := &thriftIDLProviderOptions{} tOpts.apply(opts) - svc, err := newServiceDescriptorFromPath(path, getParseMode(tOpts), tOpts.goTag, includeDirs...) + svc, err := newServiceDescriptorFromPath(path, getParseMode(tOpts), tOpts.goTag, tOpts.serviceName, includeDirs...) if err != nil { return nil, err } @@ -79,7 +79,7 @@ func NewThriftFileProviderWithDynamicgoWithOption(path string, opts []ThriftIDLP tOpts := &thriftIDLProviderOptions{} tOpts.apply(opts) parseMode := getParseMode(tOpts) - svc, err := newServiceDescriptorFromPath(path, parseMode, tOpts.goTag, includeDirs...) + svc, err := newServiceDescriptorFromPath(path, parseMode, tOpts.goTag, tOpts.serviceName, includeDirs...) if err != nil { return nil, err } @@ -90,7 +90,7 @@ func NewThriftFileProviderWithDynamicgoWithOption(path string, opts []ThriftIDLP return nil, err } handleGoTagForDynamicGo(tOpts.goTag) - dOpts := dthrift.Options{EnableThriftBase: true, ParseServiceMode: dParseMode, UseDefaultValue: true, SetOptionalBitmap: true} + dOpts := dthrift.Options{EnableThriftBase: true, ParseServiceMode: dParseMode, UseDefaultValue: true, SetOptionalBitmap: true, ServiceName: tOpts.serviceName} dsvc, err := dOpts.NewDescritorFromPath(context.Background(), path, includeDirs...) if err != nil { // fall back to the original way (without dynamicgo) @@ -105,7 +105,7 @@ func NewThriftFileProviderWithDynamicgoWithOption(path string, opts []ThriftIDLP return p, nil } -func newServiceDescriptorFromPath(path string, parseMode thrift.ParseMode, goTagOpt *goTagOption, includeDirs ...string) (*descriptor.ServiceDescriptor, error) { +func newServiceDescriptorFromPath(path string, parseMode thrift.ParseMode, goTagOpt *goTagOption, serviceName string, includeDirs ...string) (*descriptor.ServiceDescriptor, error) { tree, err := parser.ParseFile(path, includeDirs, true) if err != nil { return nil, err @@ -114,6 +114,9 @@ func newServiceDescriptorFromPath(path string, parseMode thrift.ParseMode, goTag if goTagOpt != nil { parseOpts = append(parseOpts, thrift.WithGoTagDisabled(goTagOpt.isGoTagAliasDisabled)) } + if serviceName != "" { + parseOpts = append(parseOpts, thrift.WithIDLServiceName(serviceName)) + } svc, err := thrift.Parse(tree, parseMode, parseOpts...) if err != nil { return nil, err @@ -143,11 +146,12 @@ func (p *thriftFileProvider) Option() ProviderOption { // ThriftContentProvider provide descriptor from contents type ThriftContentProvider struct { - closeOnce sync.Once - svcs chan *descriptor.ServiceDescriptor - opts *ProviderOption - parseMode thrift.ParseMode - goTagOpt *goTagOption + closeOnce sync.Once + svcs chan *descriptor.ServiceDescriptor + opts *ProviderOption + parseMode thrift.ParseMode + goTagOpt *goTagOption + serviceName string } var _ DescriptorProvider = (*ThriftContentProvider)(nil) @@ -161,12 +165,13 @@ func NewThriftContentProvider(main string, includes map[string]string, opts ...T parseMode := getParseMode(tOpts) p := &ThriftContentProvider{ - svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel - opts: &ProviderOption{DynamicGoEnabled: false}, - parseMode: parseMode, - goTagOpt: tOpts.goTag, + svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel + opts: &ProviderOption{DynamicGoEnabled: false}, + parseMode: parseMode, + goTagOpt: tOpts.goTag, + serviceName: tOpts.serviceName, } - svc, err := newServiceDescriptorFromContent(defaultMainIDLPath, main, includes, false, parseMode, tOpts.goTag) + svc, err := newServiceDescriptorFromContent(defaultMainIDLPath, main, includes, false, parseMode, tOpts.goTag, tOpts.serviceName) if err != nil { return nil, err } @@ -182,18 +187,19 @@ func NewThriftContentProviderWithDynamicGo(main string, includes map[string]stri parseMode := getParseMode(tOpts) p := &ThriftContentProvider{ - svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel - opts: &ProviderOption{DynamicGoEnabled: true}, - parseMode: parseMode, - goTagOpt: tOpts.goTag, + svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel + opts: &ProviderOption{DynamicGoEnabled: true}, + parseMode: parseMode, + goTagOpt: tOpts.goTag, + serviceName: tOpts.serviceName, } - svc, err := newServiceDescriptorFromContent(defaultMainIDLPath, main, includes, false, parseMode, tOpts.goTag) + svc, err := newServiceDescriptorFromContent(defaultMainIDLPath, main, includes, false, parseMode, tOpts.goTag, tOpts.serviceName) if err != nil { return nil, err } - p.newDynamicGoDsc(svc, defaultMainIDLPath, main, includes, parseMode, tOpts.goTag) + p.newDynamicGoDsc(svc, defaultMainIDLPath, main, includes, parseMode, tOpts.goTag, tOpts.serviceName) p.svcs <- svc return p, nil @@ -210,13 +216,16 @@ func (p *ThriftContentProvider) UpdateIDL(main string, includes map[string]strin if p.goTagOpt != nil { parseOpts = append(parseOpts, thrift.WithGoTagDisabled(p.goTagOpt.isGoTagAliasDisabled)) } + if p.serviceName != "" { + parseOpts = append(parseOpts, thrift.WithIDLServiceName(p.serviceName)) + } svc, err = thrift.Parse(tree, p.parseMode, parseOpts...) if err != nil { return err } if p.opts.DynamicGoEnabled { - p.newDynamicGoDsc(svc, defaultMainIDLPath, main, includes, p.parseMode, p.goTagOpt) + p.newDynamicGoDsc(svc, defaultMainIDLPath, main, includes, p.parseMode, p.goTagOpt, p.serviceName) } select { @@ -248,8 +257,8 @@ func (p *ThriftContentProvider) Option() ProviderOption { return *p.opts } -func (p *ThriftContentProvider) newDynamicGoDsc(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, parseMode thrift.ParseMode, goTag *goTagOption) { - if err := newDynamicGoDscFromContent(svc, path, content, includes, false, parseMode, goTag); err != nil { +func (p *ThriftContentProvider) newDynamicGoDsc(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, parseMode thrift.ParseMode, goTag *goTagOption, serviceName string) { + if err := newDynamicGoDscFromContent(svc, path, content, includes, false, parseMode, goTag, serviceName); err != nil { p.opts.DynamicGoEnabled = false } } @@ -313,11 +322,12 @@ func ParseContent(path, content string, includes map[string]string, isAbsInclude // ThriftContentWithAbsIncludePathProvider ... type ThriftContentWithAbsIncludePathProvider struct { - closeOnce sync.Once - svcs chan *descriptor.ServiceDescriptor - opts *ProviderOption - parseMode thrift.ParseMode - goTagOpt *goTagOption + closeOnce sync.Once + svcs chan *descriptor.ServiceDescriptor + opts *ProviderOption + parseMode thrift.ParseMode + goTagOpt *goTagOption + serviceName string } var _ DescriptorProvider = (*ThriftContentWithAbsIncludePathProvider)(nil) @@ -329,17 +339,18 @@ func NewThriftContentWithAbsIncludePathProvider(mainIDLPath string, includes map parseMode := getParseMode(tOpts) p := &ThriftContentWithAbsIncludePathProvider{ - svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel - opts: &ProviderOption{DynamicGoEnabled: false}, - parseMode: parseMode, - goTagOpt: tOpts.goTag, + svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel + opts: &ProviderOption{DynamicGoEnabled: false}, + parseMode: parseMode, + goTagOpt: tOpts.goTag, + serviceName: tOpts.serviceName, } mainIDLContent, ok := includes[mainIDLPath] if !ok { return nil, fmt.Errorf("miss main IDL content for main IDL path: %s", mainIDLPath) } - svc, err := newServiceDescriptorFromContent(mainIDLPath, mainIDLContent, includes, true, parseMode, tOpts.goTag) + svc, err := newServiceDescriptorFromContent(mainIDLPath, mainIDLContent, includes, true, parseMode, tOpts.goTag, tOpts.serviceName) if err != nil { return nil, err } @@ -355,22 +366,23 @@ func NewThriftContentWithAbsIncludePathProviderWithDynamicGo(mainIDLPath string, parseMode := getParseMode(tOpts) p := &ThriftContentWithAbsIncludePathProvider{ - svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel - opts: &ProviderOption{DynamicGoEnabled: true}, - parseMode: parseMode, - goTagOpt: tOpts.goTag, + svcs: make(chan *descriptor.ServiceDescriptor, 1), // unblock with buffered channel + opts: &ProviderOption{DynamicGoEnabled: true}, + parseMode: parseMode, + goTagOpt: tOpts.goTag, + serviceName: tOpts.serviceName, } mainIDLContent, ok := includes[mainIDLPath] if !ok { return nil, fmt.Errorf("miss main IDL content for main IDL path: %s", mainIDLPath) } - svc, err := newServiceDescriptorFromContent(mainIDLPath, mainIDLContent, includes, true, parseMode, tOpts.goTag) + svc, err := newServiceDescriptorFromContent(mainIDLPath, mainIDLContent, includes, true, parseMode, tOpts.goTag, tOpts.serviceName) if err != nil { return nil, err } - p.newDynamicGoDsc(svc, mainIDLPath, mainIDLContent, includes, parseMode, tOpts.goTag) + p.newDynamicGoDsc(svc, mainIDLPath, mainIDLContent, includes, parseMode, tOpts.goTag, tOpts.serviceName) p.svcs <- svc return p, nil @@ -391,13 +403,16 @@ func (p *ThriftContentWithAbsIncludePathProvider) UpdateIDL(mainIDLPath string, if p.goTagOpt != nil { parseOpts = append(parseOpts, thrift.WithGoTagDisabled(p.goTagOpt.isGoTagAliasDisabled)) } + if p.serviceName != "" { + parseOpts = append(parseOpts, thrift.WithIDLServiceName(p.serviceName)) + } svc, err = thrift.Parse(tree, p.parseMode, parseOpts...) if err != nil { return err } if p.opts.DynamicGoEnabled { - p.newDynamicGoDsc(svc, mainIDLPath, mainIDLContent, includes, p.parseMode, p.goTagOpt) + p.newDynamicGoDsc(svc, mainIDLPath, mainIDLContent, includes, p.parseMode, p.goTagOpt, p.serviceName) } // drain the channel @@ -430,8 +445,8 @@ func (p *ThriftContentWithAbsIncludePathProvider) Option() ProviderOption { return *p.opts } -func (p *ThriftContentWithAbsIncludePathProvider) newDynamicGoDsc(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, parseMode thrift.ParseMode, goTag *goTagOption) { - if err := newDynamicGoDscFromContent(svc, path, content, includes, true, parseMode, goTag); err != nil { +func (p *ThriftContentWithAbsIncludePathProvider) newDynamicGoDsc(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, parseMode thrift.ParseMode, goTag *goTagOption, serviceName string) { + if err := newDynamicGoDscFromContent(svc, path, content, includes, true, parseMode, goTag, serviceName); err != nil { p.opts.DynamicGoEnabled = false } } @@ -457,7 +472,7 @@ func getDynamicGoParseMode(parseMode thrift.ParseMode) (meta.ParseServiceMode, e } } -func newServiceDescriptorFromContent(path, content string, includes map[string]string, isAbsIncludePath bool, parseMode thrift.ParseMode, goTagOpt *goTagOption) (*descriptor.ServiceDescriptor, error) { +func newServiceDescriptorFromContent(path, content string, includes map[string]string, isAbsIncludePath bool, parseMode thrift.ParseMode, goTagOpt *goTagOption, serviceName string) (*descriptor.ServiceDescriptor, error) { tree, err := ParseContent(path, content, includes, isAbsIncludePath) if err != nil { return nil, err @@ -466,6 +481,9 @@ func newServiceDescriptorFromContent(path, content string, includes map[string]s if goTagOpt != nil { parseOpts = append(parseOpts, thrift.WithGoTagDisabled(goTagOpt.isGoTagAliasDisabled)) } + if serviceName != "" { + parseOpts = append(parseOpts, thrift.WithIDLServiceName(serviceName)) + } svc, err := thrift.Parse(tree, parseMode, parseOpts...) if err != nil { return nil, err @@ -473,14 +491,14 @@ func newServiceDescriptorFromContent(path, content string, includes map[string]s return svc, nil } -func newDynamicGoDscFromContent(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, isAbsIncludePath bool, parseMode thrift.ParseMode, goTag *goTagOption) error { +func newDynamicGoDscFromContent(svc *descriptor.ServiceDescriptor, path, content string, includes map[string]string, isAbsIncludePath bool, parseMode thrift.ParseMode, goTag *goTagOption, serviceName string) error { handleGoTagForDynamicGo(goTag) // ServiceDescriptor of dynamicgo dParseMode, err := getDynamicGoParseMode(parseMode) if err != nil { return err } - dOpts := dthrift.Options{EnableThriftBase: true, ParseServiceMode: dParseMode, UseDefaultValue: true, SetOptionalBitmap: true} + dOpts := dthrift.Options{EnableThriftBase: true, ParseServiceMode: dParseMode, UseDefaultValue: true, SetOptionalBitmap: true, ServiceName: serviceName} dsvc, err := dOpts.NewDescritorFromContent(context.Background(), path, content, includes, isAbsIncludePath) if err != nil { klog.CtxWarnf(context.Background(), "KITEX: failed to get dynamicgo service descriptor, fall back to the original way, error=%s", err) diff --git a/pkg/generic/thriftidl_provider_option.go b/pkg/generic/thriftidl_provider_option.go index 7139b84760..f143855b28 100644 --- a/pkg/generic/thriftidl_provider_option.go +++ b/pkg/generic/thriftidl_provider_option.go @@ -19,8 +19,9 @@ package generic import "github.com/cloudwego/kitex/pkg/generic/thrift" type thriftIDLProviderOptions struct { - parseMode *thrift.ParseMode - goTag *goTagOption + parseMode *thrift.ParseMode + goTag *goTagOption + serviceName string } type goTagOption struct { @@ -37,6 +38,8 @@ func (o *thriftIDLProviderOptions) apply(opts []ThriftIDLProviderOption) { } } +// WithParseMode sets the parse mode. +// NOTE: when using WithIDLServiceName at the same time, parse mode will be ignored. func WithParseMode(parseMode thrift.ParseMode) ThriftIDLProviderOption { return ThriftIDLProviderOption{F: func(opt *thriftIDLProviderOptions) { opt.parseMode = &parseMode @@ -50,3 +53,11 @@ func WithGoTagDisabled(disable bool) ThriftIDLProviderOption { } }} } + +// WithIDLServiceName specifies the target IDL service to be parsed. +// NOTE: when using this option, the specified service is prioritized, and parse mode will be ignored. +func WithIDLServiceName(serviceName string) ThriftIDLProviderOption { + return ThriftIDLProviderOption{F: func(opt *thriftIDLProviderOptions) { + opt.serviceName = serviceName + }} +} diff --git a/pkg/generic/thriftidl_provider_test.go b/pkg/generic/thriftidl_provider_test.go index 016ad05537..989cc98df0 100644 --- a/pkg/generic/thriftidl_provider_test.go +++ b/pkg/generic/thriftidl_provider_test.go @@ -550,3 +550,111 @@ func TestDefaultValue(t *testing.T) { test.Assert(t, tree.DynamicGoDsc.Functions()["Example2Method"].Request().Struct().FieldById(1).Type().Struct().FieldById(4).DefaultValue().JSONValue() == "8") p.Close() } + +func TestParseWithIDLServiceName(t *testing.T) { + path := "json_test/idl/example_multi_service.thrift" + opts := []ThriftIDLProviderOption{WithIDLServiceName("ExampleService")} + p, err := NewThriftFileProviderWithOption(path, opts) + test.Assert(t, err == nil) + tree := <-p.Provide() + test.Assert(t, tree != nil) + test.Assert(t, tree.Name == "ExampleService") + p.Close() + + p, err = NewThriftFileProviderWithDynamicgoWithOption(path, opts) + test.Assert(t, err == nil) + tree = <-p.Provide() + test.Assert(t, tree != nil) + test.Assert(t, tree.Name == "ExampleService") + test.Assert(t, tree.DynamicGoDsc.Name() == "ExampleService") + p.Close() + + content := ` + namespace go thrift + + struct Request { + 1: required string message, + } + + struct Response { + 1: required string message, + } + + service Service1 { + Response Test(1: Request req) + } + + service Service2 { + Response Test(1: Request req) + } + + service Service3 { + Response Test(1: Request req) + } + ` + + updateContent := ` + namespace go thrift + + struct Request { + 1: required string message, + } + + struct Response { + 1: required string message, + } + + service Service1 { + Response Test(1: Request req) + } + + service Service2 { + Response Test(1: Request req) + } + ` + cp, err := NewThriftContentProvider(content, nil, WithIDLServiceName("Service2")) + test.Assert(t, err == nil) + tree = <-cp.Provide() + test.Assert(t, tree.Name == "Service2") + cp.Close() + + cp, err = NewThriftContentProvider(content, nil, WithIDLServiceName("UnknownService")) + test.Assert(t, err != nil) + test.Assert(t, err.Error() == "the idl service name UnknownService is not in the idl. Please check your idl") + test.Assert(t, cp == nil) + + cp, err = NewThriftContentProviderWithDynamicGo(content, nil, WithIDLServiceName("Service1")) + test.Assert(t, err == nil) + tree = <-cp.Provide() + test.Assert(t, tree.Name == "Service1") + test.Assert(t, tree.DynamicGoDsc != nil) + test.Assert(t, tree.DynamicGoDsc.Name() == "Service1") + + err = cp.UpdateIDL(updateContent, nil) + test.Assert(t, err == nil) + tree = <-cp.Provide() + test.Assert(t, tree.Name == "Service1") + test.Assert(t, tree.DynamicGoDsc != nil) + test.Assert(t, tree.DynamicGoDsc.Name() == "Service1") + cp.Close() + + path = "a/b/main.thrift" + includes := map[string]string{path: content} + ap, err := NewThriftContentWithAbsIncludePathProvider(path, includes, WithIDLServiceName("Service2")) + test.Assert(t, err == nil) + tree = <-ap.Provide() + test.Assert(t, tree.Name == "Service2") + ap.Close() + + ap, err = NewThriftContentWithAbsIncludePathProviderWithDynamicGo(path, includes, WithIDLServiceName("Service3")) + test.Assert(t, err == nil) + tree = <-ap.Provide() + test.Assert(t, tree.Name == "Service3") + test.Assert(t, tree.DynamicGoDsc != nil) + test.Assert(t, tree.DynamicGoDsc.Name() == "Service3") + + err = ap.UpdateIDL(path, map[string]string{path: updateContent}) + test.Assert(t, err != nil) + test.Assert(t, err.Error() == "the idl service name Service3 is not in the idl. Please check your idl") + ap.Close() +} From a8d2a21c234acbfa91dfad5a406b7c911fc132c4 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Fri, 17 Jan 2025 14:18:59 +0800 Subject: [PATCH 12/35] feat(tool): use go.mod for -module (#1674) --- tool/cmd/kitex/args/args.go | 89 +++++++++++++++++++++----------- tool/cmd/kitex/args/args_test.go | 11 ++++ 2 files changed, 70 insertions(+), 30 deletions(-) diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index 5adf00575e..b609dd3249 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -16,11 +16,13 @@ package args import ( "context" + "errors" "flag" "fmt" "io" "os" "os/exec" + "path" "path/filepath" "regexp" "strconv" @@ -270,50 +272,74 @@ func (a *Arguments) checkStreamX() error { return nil } -func (a *Arguments) checkPath(curpath string) error { - pathToGo, err := exec.LookPath("go") +// refGoSrcPath returns ref path to curpath, base path is $GOPATH/src +func refGoSrcPath(curpath string) (string, bool) { + gopath, err := util.GetGOPATH() if err != nil { - return err + return "", false } - - gopath, err := util.GetGOPATH() + gosrc, err := filepath.Abs(filepath.Join(gopath, "src")) if err != nil { - return err + return "", false + } + // if curpath is NOT under gosrc + if !strings.HasPrefix(curpath, gosrc) { + return "", false } - gosrc := util.JoinPath(gopath, "src") - gosrc, err = filepath.Abs(gosrc) + ret, err := filepath.Rel(gosrc, curpath) if err != nil { - return fmt.Errorf("get GOPATH/src path failed: %s", err.Error()) + return "", false } + return ret, true +} - if strings.HasPrefix(curpath, gosrc) { - if a.PackagePrefix, err = filepath.Rel(gosrc, curpath); err != nil { - return fmt.Errorf("get GOPATH/src relpath failed: %s", err.Error()) - } - a.PackagePrefix = util.JoinPath(a.PackagePrefix, a.GenPath) - } else { - if a.ModuleName == "" { - return fmt.Errorf("outside of $GOPATH. Please specify a module name with the '-module' flag") +func (a *Arguments) checkPath(curpath string) error { + genPath := filepath.ToSlash(a.GenPath) // for PackagePrefix + usingGOPATH := false + + // Try to get ref path to $GOPARH/src for PackagePrefix + // Deprecated: to be removed in the future + if ref, ok := refGoSrcPath(curpath); ok { + usingGOPATH = true + a.PackagePrefix = path.Join(filepath.ToSlash(ref), genPath) + } + + goMod, goModPath, hasGoMod := util.SearchGoMod(curpath) + if usingGOPATH && a.ModuleName == "" && !hasGoMod { + log.Warn("[Warn] You're relying on $GOPATH for generating code.\n" + + "Please add go.mod or specify -module for module path.\n" + + "We will deprecate $GOPATH support in the near future!") + } + + if !usingGOPATH && a.ModuleName == "" { + // try to update a.ModuleName with module name from go.mod + if hasGoMod { + a.ModuleName = goMod + } else { + // case: + // * -module not set + // * not under $GOPATH/src + // * go.mod not found + return errors.New("go.mod not found. Please specify a module name with the '-module' flag") } } if a.ModuleName != "" { - module, path, ok := util.SearchGoMod(curpath) - if ok { - // go.mod exists - if module != a.ModuleName { + if hasGoMod { + if goMod != a.ModuleName { return fmt.Errorf("the module name given by the '-module' option ('%s') is not consist with the name defined in go.mod ('%s' from %s)", - a.ModuleName, module, path) + a.ModuleName, goMod, goModPath) } - if a.PackagePrefix, err = filepath.Rel(path, curpath); err != nil { - return fmt.Errorf("get package prefix failed: %s", err.Error()) + refPath, err := filepath.Rel(goModPath, curpath) + if err != nil { + return fmt.Errorf("get package prefix failed: %w", err) } - a.PackagePrefix = util.JoinPath(a.ModuleName, a.PackagePrefix, a.GenPath) + a.PackagePrefix = path.Join(a.ModuleName, filepath.ToSlash(refPath), genPath) } else { - if err = initGoMod(pathToGo, a.ModuleName); err != nil { - return fmt.Errorf("init go mod failed: %s", err.Error()) + if err := initGoMod(a.ModuleName); err != nil { + return fmt.Errorf("init go mod failed: %w", err) } - a.PackagePrefix = util.JoinPath(a.ModuleName, a.GenPath) + a.PackagePrefix = path.Join(a.ModuleName, genPath) } } @@ -541,11 +567,14 @@ func LookupTool(idlType, compilerPath string) string { return path } -func initGoMod(pathToGo, module string) error { +func initGoMod(module string) error { if util.Exists("go.mod") { return nil } - + pathToGo, err := exec.LookPath("go") + if err != nil { + return err + } cmd := &exec.Cmd{ Path: pathToGo, Args: []string{"go", "mod", "init", module}, diff --git a/tool/cmd/kitex/args/args_test.go b/tool/cmd/kitex/args/args_test.go index 4732284527..93d0675ea4 100644 --- a/tool/cmd/kitex/args/args_test.go +++ b/tool/cmd/kitex/args/args_test.go @@ -17,6 +17,8 @@ package args import ( + "os" + "path/filepath" "testing" "github.com/cloudwego/kitex/internal/test" @@ -52,3 +54,12 @@ func Test_versionSatisfied(t *testing.T) { test.Assert(t, !versionSatisfied("1.2.3", "1.2.4")) }) } + +func Test_refGoSrcPath(t *testing.T) { + gopath, _ := os.Getwd() + t.Setenv("GOPATH", gopath) + pkgpath := filepath.Join(gopath, "src", "github.com", "cloudwego", "kitex") + ret, ok := refGoSrcPath(pkgpath) + test.Assert(t, ok) + test.Assert(t, filepath.ToSlash(ret) == "github.com/cloudwego/kitex") +} From 789cc11b1a1faedcc747181d49ba1a683fa6b97d Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Mon, 20 Jan 2025 14:33:46 +0800 Subject: [PATCH 13/35] fix(gRPC): ignore triggered actively connection error (#1677) --- pkg/remote/trans/nphttp2/grpc/controlbuf.go | 2 +- pkg/remote/trans/nphttp2/grpc/http2_client.go | 4 ++-- pkg/remote/trans/nphttp2/grpc/transport.go | 23 ++++++++++++++++++- .../trans/nphttp2/grpc/transport_test.go | 12 ++++++++++ 4 files changed, 37 insertions(+), 4 deletions(-) diff --git a/pkg/remote/trans/nphttp2/grpc/controlbuf.go b/pkg/remote/trans/nphttp2/grpc/controlbuf.go index b4edd6d7ce..417200b099 100644 --- a/pkg/remote/trans/nphttp2/grpc/controlbuf.go +++ b/pkg/remote/trans/nphttp2/grpc/controlbuf.go @@ -560,7 +560,7 @@ const minBatchSize = 1000 // if the batch size is too low to give stream goroutines a chance to fill it up. func (l *loopyWriter) run(remoteAddr string) (err error) { defer func() { - if err == ErrConnClosing { + if isIgnorable(err) { // Don't log ErrConnClosing as error since it happens // 1. When the connection is closed by some other known issue. // 2. User closed the connection. diff --git a/pkg/remote/trans/nphttp2/grpc/http2_client.go b/pkg/remote/trans/nphttp2/grpc/http2_client.go index 091729f8f5..f2d6a4ccfd 100644 --- a/pkg/remote/trans/nphttp2/grpc/http2_client.go +++ b/pkg/remote/trans/nphttp2/grpc/http2_client.go @@ -664,7 +664,7 @@ func (t *http2Client) GracefulClose() { active := len(t.activeStreams) t.mu.Unlock() if active == 0 { - t.Close(connectionErrorf(true, nil, "no active streams left to process while draining")) + t.Close(connectionErrorfWithIgnorable(true, nil, "no active streams left to process while draining")) return } t.controlBuf.put(&incomingGoAway{}) @@ -961,7 +961,7 @@ func (t *http2Client) handleGoAway(f *grpcframe.GoAwayFrame) { active := len(t.activeStreams) if active <= 0 { t.mu.Unlock() - t.Close(connectionErrorf(true, nil, "received goaway and there are no active streams")) + t.Close(connectionErrorfWithIgnorable(true, nil, "received goaway and there are no active streams")) return } diff --git a/pkg/remote/trans/nphttp2/grpc/transport.go b/pkg/remote/trans/nphttp2/grpc/transport.go index 912ca87b5a..f27581de4b 100644 --- a/pkg/remote/trans/nphttp2/grpc/transport.go +++ b/pkg/remote/trans/nphttp2/grpc/transport.go @@ -748,12 +748,21 @@ func connectionErrorf(temp bool, e error, format string, a ...interface{}) Conne } } +// connectionErrorfWithIgnorable creates an ConnectionError with the specified error description and isIgnorable == true +func connectionErrorfWithIgnorable(temp bool, e error, format string, a ...interface{}) ConnectionError { + connErr := connectionErrorf(temp, e, format, a...) + connErr.isIgnorable = true + return connErr +} + // ConnectionError is an error that results in the termination of the // entire connection and the retry of all the active streams. type ConnectionError struct { Desc string temp bool err error + // isIgnorable indicates whether this error is triggered by Kitex initiative and could be ignored. + isIgnorable bool } func (e ConnectionError) Error() string { @@ -781,9 +790,21 @@ func (e ConnectionError) Code() int32 { return int32(codes.Unavailable) } +func (e ConnectionError) ignorable() bool { + return e.isIgnorable +} + +// isIgnorable checks if the error is ignorable. +func isIgnorable(rawErr error) bool { + if err, ok := rawErr.(ConnectionError); ok { + return err.ignorable() + } + return false +} + var ( // ErrConnClosing indicates that the transport is closing. - ErrConnClosing = connectionErrorf(true, nil, "transport is closing") + ErrConnClosing = connectionErrorfWithIgnorable(true, nil, "transport is closing") // errStreamDone is returned from write at the client side to indicate application // layer of an error. diff --git a/pkg/remote/trans/nphttp2/grpc/transport_test.go b/pkg/remote/trans/nphttp2/grpc/transport_test.go index aada4f92aa..9f584386bb 100644 --- a/pkg/remote/trans/nphttp2/grpc/transport_test.go +++ b/pkg/remote/trans/nphttp2/grpc/transport_test.go @@ -2125,3 +2125,15 @@ func TestTlsAppendH2ToALPNProtocols(t *testing.T) { appended = tlsAppendH2ToALPNProtocols(appended) test.Assert(t, len(appended) == 1) } + +func Test_isIgnorable(t *testing.T) { + test.Assert(t, !isIgnorable(nil)) + + err := connectionErrorfWithIgnorable(true, nil, "ignorable") + test.Assert(t, isIgnorable(err)) + + err = connectionErrorf(true, nil, "not ignorable") + test.Assert(t, !isIgnorable(err)) + + test.Assert(t, isIgnorable(ErrConnClosing)) +} From 1de9e03154ac228838c331e59e3f6e6716a85055 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Mon, 20 Jan 2025 15:18:02 +0800 Subject: [PATCH 14/35] chore(tool): clearer log pkg usage (#1678) --- tool/cmd/kitex/args/args.go | 15 +++-- tool/cmd/kitex/main.go | 13 ++-- tool/cmd/kitex/sdk/kitex_sdk.go | 20 +++--- tool/cmd/kitex/utils/utils.go | 2 +- tool/cmd/kitex/versions/dependencies.go | 2 +- tool/internal_pkg/generator/completor.go | 8 ++- .../internal_pkg/generator/custom_template.go | 2 +- tool/internal_pkg/generator/generator.go | 2 +- tool/internal_pkg/log/log.go | 67 ++++++++++++++----- tool/internal_pkg/pluginmode/protoc/plugin.go | 7 +- tool/internal_pkg/pluginmode/protoc/protoc.go | 4 +- .../pluginmode/thriftgo/convertor.go | 24 ++----- 12 files changed, 93 insertions(+), 73 deletions(-) diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index b609dd3249..9a0aa7c3c3 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -29,7 +29,6 @@ import ( "strings" "time" - "github.com/cloudwego/kitex/pkg/klog" "github.com/cloudwego/kitex/tool/internal_pkg/generator" "github.com/cloudwego/kitex/tool/internal_pkg/log" "github.com/cloudwego/kitex/tool/internal_pkg/pluginmode/protoc" @@ -306,7 +305,7 @@ func (a *Arguments) checkPath(curpath string) error { goMod, goModPath, hasGoMod := util.SearchGoMod(curpath) if usingGOPATH && a.ModuleName == "" && !hasGoMod { - log.Warn("[Warn] You're relying on $GOPATH for generating code.\n" + + log.Warn("You're relying on $GOPATH for generating code.\n" + "Please add go.mod or specify -module for module path.\n" + "We will deprecate $GOPATH support in the near future!") } @@ -370,7 +369,8 @@ func (a *Arguments) BuildCmd(out io.Writer) (*exec.Cmd, error) { } } - kas := strings.Join(a.Config.Pack(), ",") + configkv := a.Config.Pack() + kas := strings.Join(configkv, ",") cmd := &exec.Cmd{ Path: LookupTool(a.IDLType, a.CompilerPath), Stdin: os.Stdin, @@ -453,7 +453,8 @@ func (a *Arguments) BuildCmd(out io.Writer) (*exec.Cmd, error) { cmd.Args = append(cmd.Args, a.IDL) } - log.Info(strings.ReplaceAll(strings.Join(cmd.Args, " "), kas, fmt.Sprintf("%q", kas))) + log.Debugf("cmd.Args %v", cmd.Args) + log.Debugf("config pairs %v", configkv) return cmd, nil } @@ -525,11 +526,11 @@ func versionSatisfied(current, required string) bool { var currentSeg, minimalSeg int var err error if currentSeg, err = strconv.Atoi(currentSegments[i]); err != nil { - klog.Warnf("invalid current version: %s, seg=%v, err=%v", current, currentSegments[i], err) + log.Warnf("invalid current version: %s, seg=%v, err=%v", current, currentSegments[i], err) return false } if minimalSeg, err = strconv.Atoi(requiredSegments[i]); err != nil { - klog.Warnf("invalid required version: %s, seg=%v, err=%v", required, requiredSegments[i], err) + log.Warnf("invalid required version: %s, seg=%v, err=%v", required, requiredSegments[i], err) return false } if currentSeg > minimalSeg { @@ -551,7 +552,7 @@ func LookupTool(idlType, compilerPath string) string { tool = "protoc" } if compilerPath != "" { - log.Infof("Will use the specified %s: %s\n", tool, compilerPath) + log.Debugf("Will use the specified %s: %s\n", tool, compilerPath) return compilerPath } diff --git a/tool/cmd/kitex/main.go b/tool/cmd/kitex/main.go index a1530837c7..de4dcf0c44 100644 --- a/tool/cmd/kitex/main.go +++ b/tool/cmd/kitex/main.go @@ -16,6 +16,7 @@ package main import ( "bytes" + "errors" "flag" "os" "path/filepath" @@ -56,7 +57,7 @@ func init() { Version: "v0.11.0", }, ); err != nil { - log.Warn(err) + log.Error(err) os.Exit(versions.CompatibilityCheckExitCode) } } @@ -75,17 +76,17 @@ func main() { curpath, err := filepath.Abs(".") if err != nil { - log.Warn("Get current path failed:", err.Error()) + log.Errorf("Get current path failed: %s", err) os.Exit(1) } // run as kitex err = args.ParseArgs(kitex.Version, curpath, os.Args[1:]) if err != nil { - if err.Error() != "flag: help requested" { - log.Warn(err.Error()) - os.Exit(2) + if errors.Is(err, flag.ErrHelp) { + os.Exit(0) } - os.Exit(0) + log.Error(err) + os.Exit(2) } if !args.NoDependencyCheck { // check dependency compatibility between kitex cmd tool and dependency in go.mod diff --git a/tool/cmd/kitex/sdk/kitex_sdk.go b/tool/cmd/kitex/sdk/kitex_sdk.go index 1cd6e3d445..4411a06aab 100644 --- a/tool/cmd/kitex/sdk/kitex_sdk.go +++ b/tool/cmd/kitex/sdk/kitex_sdk.go @@ -16,6 +16,7 @@ package sdk import ( "bytes" + "errors" "flag" "fmt" "os/exec" @@ -33,7 +34,7 @@ import ( var args kargs.Arguments -var errExitZero = fmt.Errorf("os.Exit(0)") +var errExitZero = errors.New("os.Exit(0)") func init() { var queryVersion bool @@ -55,7 +56,7 @@ func init() { func RunKitexTool(wd string, plugins []plugin.SDKPlugin, kitexArgs ...string) error { kitexPlugin, err := GetKiteXSDKPlugin(wd, kitexArgs) if err != nil { - if err.Error() == "flag: help requested" || err == errExitZero { + if errors.Is(err, flag.ErrHelp) || errors.Is(err, errExitZero) { return nil } return err @@ -101,16 +102,11 @@ func InvokeThriftgoBySDK(pwd string, cmd *exec.Cmd) (err error) { kitexPlugin.Pwd = pwd - s := []plugin.SDKPlugin{kitexPlugin} - - err = sdk.RunThriftgoAsSDK(pwd, s, kitexPlugin.GetThriftgoParameters()...) - // when execute thriftgo as function, log will be unexpectedly replaced (for old code by design), so we have to change it back. - log.SetDefaultLogger(log.Logger{ - Println: fmt.Fprintln, - Printf: fmt.Fprintf, - }) - - return err + l := log.DefaultLogger() // pluginmode/thriftgo/convertor.go will change the logger + defer log.SetDefaultLogger(l) // revert it back + return sdk.RunThriftgoAsSDK(pwd, + []plugin.SDKPlugin{kitexPlugin}, + kitexPlugin.GetThriftgoParameters()...) } type KiteXSDKPlugin struct { diff --git a/tool/cmd/kitex/utils/utils.go b/tool/cmd/kitex/utils/utils.go index 3b257d462e..f24378aefc 100644 --- a/tool/cmd/kitex/utils/utils.go +++ b/tool/cmd/kitex/utils/utils.go @@ -24,7 +24,7 @@ import ( ) func OnKitexToolNormalExit(args kargs.Arguments) { - log.Warn("Code Generation is Done!") + log.Info("Code Generation is Done!") if args.IDLType == "thrift" { cmd := "go mod edit -replace github.com/apache/thrift=github.com/apache/thrift@v0.13.0" diff --git a/tool/cmd/kitex/versions/dependencies.go b/tool/cmd/kitex/versions/dependencies.go index c552b83dd5..2113d1c81a 100644 --- a/tool/cmd/kitex/versions/dependencies.go +++ b/tool/cmd/kitex/versions/dependencies.go @@ -78,7 +78,7 @@ func DefaultCheckDependencyAndProcess() error { } res, shouldExit := defaultParseCheckResult(cr) if res != "" { - log.Warn(res) + log.Info(res) } if shouldExit { return errors.New("kitex cmd tool dependency compatibility check failed") diff --git a/tool/internal_pkg/generator/completor.go b/tool/internal_pkg/generator/completor.go index 154b7caac0..03d07814c3 100644 --- a/tool/internal_pkg/generator/completor.go +++ b/tool/internal_pkg/generator/completor.go @@ -83,7 +83,7 @@ func (c *completer) compare(pkg *ast.Package) []*MethodInfo { } } if !have { - log.Infof("[complete handler] add '%s' to handler.go\n", m.Name) + log.Debugf("[complete handler] add '%s' to handler.go\n", m.Name) newMethods = append(newMethods, m) } } @@ -159,8 +159,10 @@ func (c *completer) process(w io.Writer) error { fset := token.NewFileSet() pkgs, err := parser.ParseDir(fset, filepath.Dir(c.handlerPath), nil, parser.ParseComments) if err != nil { - err = fmt.Errorf("go/parser failed to parse the main package: %w", err) - log.Warn("NOTICE: This is not a bug. We cannot add new methods to handler.go because your codes failed to compile. Fix the compile errors and try again.\n%s", err.Error()) + log.Warnf("This is not a bug. \n"+ + "We cannot add new methods to handler.go because your codes failed to compile. \n"+ + "Fix the compile errors and try again.\n"+ + "go/parser err: %s", err) return err } main, ok := pkgs["main"] diff --git a/tool/internal_pkg/generator/custom_template.go b/tool/internal_pkg/generator/custom_template.go index 3f364589db..4b790e6192 100644 --- a/tool/internal_pkg/generator/custom_template.go +++ b/tool/internal_pkg/generator/custom_template.go @@ -129,7 +129,7 @@ func (c *customGenerator) commonGenerate(tpl *Template) error { filePath := filepath.Join(c.basePath, renderPath) update := util.Exists(filePath) if update && updateType(tpl.UpdateBehavior.Type) == skip { - log.Infof("skip generate file %s", tpl.Path) + log.Debugf("skip generate file %s", tpl.Path) return nil } var f *File diff --git a/tool/internal_pkg/generator/generator.go b/tool/internal_pkg/generator/generator.go index 885b1f8945..d150ca7bec 100644 --- a/tool/internal_pkg/generator/generator.go +++ b/tool/internal_pkg/generator/generator.go @@ -377,7 +377,7 @@ func (g *generator) GenerateMainPackage(pkg *PackageInfo) (fs []*File, err error } for _, t := range tasks { if util.Exists(t.Path) { - log.Info(t.Path, "exists. Skipped.") + log.Debug(t.Path, "exists. Skipped.") continue } g.setImports(t.Name, pkg) diff --git a/tool/internal_pkg/log/log.go b/tool/internal_pkg/log/log.go index 699e8dd66b..106d5bbb9d 100644 --- a/tool/internal_pkg/log/log.go +++ b/tool/internal_pkg/log/log.go @@ -16,7 +16,7 @@ package log import ( "fmt" - "io" + "log" "os" ) @@ -24,14 +24,23 @@ import ( var Verbose bool // Logger . -type Logger struct { - Println func(w io.Writer, a ...interface{}) (n int, err error) - Printf func(w io.Writer, format string, a ...interface{}) (n int, err error) +type Logger interface { + Printf(format string, a ...interface{}) } -var defaultLogger = Logger{ - Println: fmt.Fprintln, - Printf: fmt.Fprintf, +// LoggerFunc implements Logger +type LoggerFunc func(format string, a ...interface{}) + +func (f LoggerFunc) Printf(format string, a ...interface{}) { + f(format, a...) +} + +// must use os.Stderr, os.Stdout is used by plugin for output +var defaultLogger Logger = log.New(os.Stderr, "", 0) + +// DefaultLogger ... +func DefaultLogger() Logger { + return defaultLogger } // SetDefaultLogger sets the default logger. @@ -39,26 +48,48 @@ func SetDefaultLogger(l Logger) { defaultLogger = l } -// Warn . +// Error ... +func Error(v ...interface{}) { + defaultLogger.Printf("[ERROR] %s", fmt.Sprintln(v...)) +} + +// Errorf ... +func Errorf(format string, v ...interface{}) { + defaultLogger.Printf("[ERROR] "+format, v...) +} + +// Warn ... func Warn(v ...interface{}) { - defaultLogger.Println(os.Stderr, v...) + defaultLogger.Printf("[WARN] %s", fmt.Sprintln(v...)) } -// Warnf . +// Warnf ... func Warnf(format string, v ...interface{}) { - defaultLogger.Printf(os.Stderr, format, v...) + defaultLogger.Printf("[WARN] "+format, v...) } -// Info . +// Info ... func Info(v ...interface{}) { - if Verbose { - defaultLogger.Println(os.Stderr, v...) - } + defaultLogger.Printf("[INFO] %s", fmt.Sprintln(v...)) } -// Infof . +// Infof ... func Infof(format string, v ...interface{}) { - if Verbose { - defaultLogger.Printf(os.Stderr, format, v...) + defaultLogger.Printf("[INFO] "+format, v...) +} + +// Debug ... +func Debug(v ...interface{}) { + if !Verbose { + return + } + defaultLogger.Printf("[DEBUG] %s", fmt.Sprintln(v...)) +} + +// Debugf ... +func Debugf(format string, v ...interface{}) { + if !Verbose { + return } + defaultLogger.Printf("[DEBUG] "+format, v...) } diff --git a/tool/internal_pkg/pluginmode/protoc/plugin.go b/tool/internal_pkg/pluginmode/protoc/plugin.go index a7fb50fc3f..71700645ce 100644 --- a/tool/internal_pkg/pluginmode/protoc/plugin.go +++ b/tool/internal_pkg/pluginmode/protoc/plugin.go @@ -122,11 +122,12 @@ func (pp *protocPlugin) GenerateFile(gen *protogen.Plugin, file *protogen.File) } gopkg := file.Proto.GetOptions().GetGoPackage() if !strings.HasPrefix(gopkg, pp.PackagePrefix) { - log.Warnf("[WARN] %q is skipped because its import path %q is not located in ./kitex_gen. Change the go_package option or use '--protobuf M%s=A-Import-Path-In-kitex_gen' to override it if you want this file to be generated under kitex_gen.\n", + log.Warnf("%q is skipped because its import path %q is not located in ./kitex_gen.\n"+ + "Change the go_package option or use '--protobuf M%s=A-Import-Path-In-kitex_gen' to override it if you want this file to be generated under kitex_gen.\n", file.Proto.GetName(), gopkg, file.Proto.GetName()) return } - log.Infof("[INFO] Generate %q at %q\n", file.Proto.GetName(), gopkg) + log.Debugf("Generate %q at %q\n", file.Proto.GetName(), gopkg) if parts := strings.Split(gopkg, ";"); len(parts) > 1 { gopkg = parts[0] // remove package alias from file path @@ -320,7 +321,7 @@ func (pp *protocPlugin) convertTypes(file *protogen.File) (ss []*generator.Servi mm := make(map[string]*generator.MethodInfo) for _, m := range methods { if _, ok := mm[m.Name]; ok { - log.Warnf("[WARN] combine service method %s in %s conflicts with %s in %s\n", + log.Warnf("combine service method %s in %s conflicts with %s in %s\n", m.Name, m.ServiceName, m.Name, mm[m.Name].ServiceName) return } diff --git a/tool/internal_pkg/pluginmode/protoc/protoc.go b/tool/internal_pkg/pluginmode/protoc/protoc.go index 28c53c4588..8a3e7c6e05 100644 --- a/tool/internal_pkg/pluginmode/protoc/protoc.go +++ b/tool/internal_pkg/pluginmode/protoc/protoc.go @@ -93,7 +93,7 @@ func GenKitex(req *pluginpb.CodeGeneratorRequest, opts protogen.Options) (*plugi gopkg, ok := pp.importPaths[f.GetName()] if ok { f.Options.GoPackage = &gopkg - log.Infof("[INFO] option specified import path for %q: %q\n", f.GetName(), gopkg) + log.Debugf("option specified import path for %q: %q\n", f.GetName(), gopkg) } else { if f.Options == nil || f.Options.GoPackage == nil { return nil, fmt.Errorf("ERROR: go_package is missing in proto file %q", f.GetName()) @@ -102,7 +102,7 @@ func GenKitex(req *pluginpb.CodeGeneratorRequest, opts protogen.Options) (*plugi } if path, ok := pe.getImportPath(gopkg); ok { f.Options.GoPackage = &path - log.Infof("[INFO] update import path for %q: %q -> %q\n", f.GetName(), gopkg, path) + log.Debugf("update import path for %q: %q -> %q\n", f.GetName(), gopkg, path) } } diff --git a/tool/internal_pkg/pluginmode/thriftgo/convertor.go b/tool/internal_pkg/pluginmode/thriftgo/convertor.go index 2137eaaa2d..072d08616f 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/convertor.go +++ b/tool/internal_pkg/pluginmode/thriftgo/convertor.go @@ -17,7 +17,6 @@ package thriftgo import ( "fmt" "go/format" - "io" "os" "path/filepath" "regexp" @@ -31,7 +30,7 @@ import ( "github.com/cloudwego/thriftgo/semantic" "github.com/cloudwego/kitex/tool/internal_pkg/generator" - internal_log "github.com/cloudwego/kitex/tool/internal_pkg/log" + "github.com/cloudwego/kitex/tool/internal_pkg/log" "github.com/cloudwego/kitex/tool/internal_pkg/util" "github.com/cloudwego/kitex/transport" ) @@ -80,20 +79,9 @@ func (c *converter) initLogs() backend.LogFunc { lf.Info = lf.Warn } - internal_log.SetDefaultLogger(internal_log.Logger{ - Println: func(w io.Writer, a ...interface{}) (n int, err error) { - if w != os.Stdout || c.Config.Verbose { - c.Warnings = append(c.Warnings, fmt.Sprint(a...)) - } - return 0, nil - }, - Printf: func(w io.Writer, format string, a ...interface{}) (n int, err error) { - if w != os.Stdout || c.Config.Verbose { - c.Warnings = append(c.Warnings, fmt.Sprintf(format, a...)) - } - return 0, nil - }, - }) + log.SetDefaultLogger(log.LoggerFunc(func(format string, a ...interface{}) { + c.Warnings = append(c.Warnings, fmt.Sprintf(format, a...)) + })) return lf } @@ -516,13 +504,13 @@ func (c *converter) persist(res *plugin.Response) error { content := []byte(c.Content) if filepath.Ext(full) == ".go" { if formatted, err := format.Source([]byte(c.Content)); err != nil { - internal_log.Warn(fmt.Sprintf("Failed to format %s: %s", full, err.Error())) + log.Warnf("Failed to format %s: %s", full, err) } else { content = formatted } } - internal_log.Info("Write", full) + log.Debug("Write", full) path := filepath.Dir(full) if err := os.MkdirAll(path, 0o755); err != nil && !os.IsExist(err) { return fmt.Errorf("failed to create path '%s': %w", path, err) From 4f3c3fcb18a6fe092b1df22f3f44b40a46a1aba6 Mon Sep 17 00:00:00 2001 From: Jayant Date: Wed, 22 Jan 2025 15:29:13 +0800 Subject: [PATCH 15/35] optimize(exit): server graceful shutdown logic to avoid EOF when idle connections receive new requests after being closed (#1681) --- pkg/remote/trans/default_server_handler.go | 38 ++++-- pkg/remote/trans/netpoll/trans_server.go | 9 ++ pkg/retry/failure_test.go | 2 +- pkg/transmeta/ttheader.go | 8 ++ pkg/transmeta/ttheader_test.go | 2 +- server/server_test.go | 136 +++++++++++---------- 6 files changed, 117 insertions(+), 78 deletions(-) diff --git a/pkg/remote/trans/default_server_handler.go b/pkg/remote/trans/default_server_handler.go index 3d91c206d4..0b557428e9 100644 --- a/pkg/remote/trans/default_server_handler.go +++ b/pkg/remote/trans/default_server_handler.go @@ -22,6 +22,7 @@ import ( "fmt" "net" "runtime/debug" + "sync/atomic" "github.com/cloudwego/kitex/pkg/endpoint" "github.com/cloudwego/kitex/pkg/kerrors" @@ -49,13 +50,14 @@ func NewDefaultSvrTransHandler(opt *remote.ServerOption, ext Extension) (remote. } type svrTransHandler struct { - opt *remote.ServerOption - svcSearcher remote.ServiceSearcher - targetSvcInfo *serviceinfo.ServiceInfo - inkHdlFunc endpoint.Endpoint - codec remote.Codec - transPipe *remote.TransPipeline - ext Extension + opt *remote.ServerOption + svcSearcher remote.ServiceSearcher + targetSvcInfo *serviceinfo.ServiceInfo + inkHdlFunc endpoint.Endpoint + codec remote.Codec + transPipe *remote.TransPipeline + ext Extension + inGracefulShutdown uint32 } // Write implements the remote.ServerTransHandler interface. @@ -123,13 +125,22 @@ func (t *svrTransHandler) Read(ctx context.Context, conn net.Conn, recvMsg remot } func (t *svrTransHandler) newCtxWithRPCInfo(ctx context.Context, conn net.Conn) (context.Context, rpcinfo.RPCInfo) { + var ri rpcinfo.RPCInfo if rpcinfo.PoolEnabled() { // reuse per-connection rpcinfo - return ctx, rpcinfo.GetRPCInfo(ctx) + ri = rpcinfo.GetRPCInfo(ctx) // delayed reinitialize for faster response + } else { + // new rpcinfo if reuse is disabled + ri = t.opt.InitOrResetRPCInfoFunc(nil, conn.RemoteAddr()) + ctx = rpcinfo.NewCtxWithRPCInfo(ctx, ri) } - // new rpcinfo if reuse is disabled - ri := t.opt.InitOrResetRPCInfoFunc(nil, conn.RemoteAddr()) - return rpcinfo.NewCtxWithRPCInfo(ctx, ri), ri + if atomic.LoadUint32(&t.inGracefulShutdown) == 1 { + // If server is in graceful shutdown status, mark connection reset flag to all responses to let client close the connections. + if ei := rpcinfo.AsTaggable(ri.To()); ei != nil { + ei.SetTag(rpcinfo.ConnResetTag, "1") + } + } + return ctx, ri } // OnRead implements the remote.ServerTransHandler interface. @@ -348,6 +359,11 @@ func (t *svrTransHandler) finishProfiler(ctx context.Context) { t.opt.Profiler.Untag(ctx) } +func (t *svrTransHandler) GracefulShutdown(ctx context.Context) error { + atomic.StoreUint32(&t.inGracefulShutdown, 1) + return nil +} + func getRemoteInfo(ri rpcinfo.RPCInfo, conn net.Conn) (string, net.Addr) { rAddr := conn.RemoteAddr() if ri == nil { diff --git a/pkg/remote/trans/netpoll/trans_server.go b/pkg/remote/trans/netpoll/trans_server.go index 527022f6f9..d2ca8f86c0 100644 --- a/pkg/remote/trans/netpoll/trans_server.go +++ b/pkg/remote/trans/netpoll/trans_server.go @@ -25,6 +25,7 @@ import ( "runtime/debug" "sync" "syscall" + "time" "github.com/cloudwego/netpoll" @@ -119,6 +120,14 @@ func (ts *transServer) Shutdown() (err error) { if err != nil { klog.Warnf("KITEX: server graceful shutdown error: %v", err) } + // 3. wait some time to receive requests before closing idle conns + /* + When the netpoll eventloop shutdown, all idle connections will be closed. + At this time, these connections may just receive requests, and then the peer side will report an EOF error. + To reduce such cases, wait for some time to try to receive these requests as much as possible, + so that the closing of connections can be controlled by the upper-layer protocol and the EOF problem can be reduced. + */ + time.Sleep(100 * time.Millisecond) } } if ts.evl != nil { diff --git a/pkg/retry/failure_test.go b/pkg/retry/failure_test.go index 1c63c2b0a9..055da2f6a6 100644 --- a/pkg/retry/failure_test.go +++ b/pkg/retry/failure_test.go @@ -80,7 +80,7 @@ func TestFixedBackOff_Wait(t *testing.T) { bk.Wait(1) waitTime := time.Since(startTime) test.Assert(t, time.Millisecond*fix <= waitTime) - test.Assert(t, waitTime < time.Millisecond*(fix+5)) + test.Assert(t, waitTime < time.Millisecond*(fix*2)) } func TestFixedBackOff_String(t *testing.T) { diff --git a/pkg/transmeta/ttheader.go b/pkg/transmeta/ttheader.go index 3b18537a58..b37519c1de 100644 --- a/pkg/transmeta/ttheader.go +++ b/pkg/transmeta/ttheader.go @@ -122,6 +122,11 @@ func (ch *clientTTHeaderHandler) ReadMeta(ctx context.Context, msg remote.Messag if setter, ok := ri.Invocation().(rpcinfo.InvocationSetter); ok && bizErr != nil { setter.SetBizStatusErr(bizErr) } + if val, ok := strInfo[transmeta.HeaderConnectionReadyToReset]; ok { + if ei := rpcinfo.AsTaggable(ri.To()); ei != nil { + ei.SetTag(rpcinfo.ConnResetTag, val) + } + } return ctx, nil } @@ -190,6 +195,9 @@ func (sh *serverTTHeaderHandler) WriteMeta(ctx context.Context, msg remote.Messa strInfo[bizExtra], _ = utils.Map2JSONStr(bizErr.BizExtra()) } } + if val, ok := ri.To().Tag(rpcinfo.ConnResetTag); ok { + strInfo[transmeta.HeaderConnectionReadyToReset] = val + } return ctx, nil } diff --git a/pkg/transmeta/ttheader_test.go b/pkg/transmeta/ttheader_test.go index 54608d18f4..9bc973dc1a 100644 --- a/pkg/transmeta/ttheader_test.go +++ b/pkg/transmeta/ttheader_test.go @@ -120,7 +120,7 @@ func TestTTHeaderServerReadMetainfo(t *testing.T) { func TestTTHeaderServerWriteMetainfo(t *testing.T) { ctx := context.Background() - ri := rpcinfo.NewRPCInfo(nil, nil, rpcinfo.NewInvocation("", ""), + ri := rpcinfo.NewRPCInfo(nil, rpcinfo.NewEndpointInfo("", "mock", nil, nil), rpcinfo.NewInvocation("", ""), rpcinfo.NewRPCConfig(), rpcinfo.NewRPCStats()) msg := remote.NewMessage(nil, mocks.ServiceInfo(), ri, remote.Call, remote.Client) diff --git a/server/server_test.go b/server/server_test.go index 14f2500501..1b479027cf 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -979,80 +979,86 @@ func TestInvokeHandlerPanic(t *testing.T) { } func TestRegisterService(t *testing.T) { - svr := NewServer() - time.AfterFunc(time.Second, func() { - err := svr.Stop() - test.Assert(t, err == nil, err) - }) + { + svr := NewServer() + time.AfterFunc(time.Second, func() { + err := svr.Stop() + test.Assert(t, err == nil, err) + }) - svr.Run() + svr.Run() - test.PanicAt(t, func() { - _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) - }, func(err interface{}) bool { - if errMsg, ok := err.(string); ok { - return strings.Contains(errMsg, "server is running") - } - return true - }) - svr.Stop() + test.PanicAt(t, func() { + _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) + }, func(err interface{}) bool { + if errMsg, ok := err.(string); ok { + return strings.Contains(errMsg, "server is running") + } + return true + }) + svr.Stop() + } - svr = NewServer() - time.AfterFunc(time.Second, func() { - err := svr.Stop() - test.Assert(t, err == nil, err) - }) + { + svr := NewServer() + time.AfterFunc(time.Second, func() { + err := svr.Stop() + test.Assert(t, err == nil, err) + }) - test.PanicAt(t, func() { - _ = svr.RegisterService(nil, mocks.MyServiceHandler()) - }, func(err interface{}) bool { - if errMsg, ok := err.(string); ok { - return strings.Contains(errMsg, "svcInfo is nil") - } - return true - }) + test.PanicAt(t, func() { + _ = svr.RegisterService(nil, mocks.MyServiceHandler()) + }, func(err interface{}) bool { + if errMsg, ok := err.(string); ok { + return strings.Contains(errMsg, "svcInfo is nil") + } + return true + }) - test.PanicAt(t, func() { - _ = svr.RegisterService(mocks.ServiceInfo(), nil) - }, func(err interface{}) bool { - if errMsg, ok := err.(string); ok { - return strings.Contains(errMsg, "handler is nil") - } - return true - }) + test.PanicAt(t, func() { + _ = svr.RegisterService(mocks.ServiceInfo(), nil) + }, func(err interface{}) bool { + if errMsg, ok := err.(string); ok { + return strings.Contains(errMsg, "handler is nil") + } + return true + }) - test.PanicAt(t, func() { - _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler(), WithFallbackService()) - _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) - }, func(err interface{}) bool { - if errMsg, ok := err.(string); ok { - return strings.Contains(errMsg, "Service[MockService] is already defined") - } - return true - }) + test.PanicAt(t, func() { + _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler(), WithFallbackService()) + _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) + }, func(err interface{}) bool { + if errMsg, ok := err.(string); ok { + return strings.Contains(errMsg, "Service[MockService] is already defined") + } + return true + }) - test.PanicAt(t, func() { - _ = svr.RegisterService(mocks.Service2Info(), mocks.MyServiceHandler(), WithFallbackService()) - }, func(err interface{}) bool { - if errMsg, ok := err.(string); ok { - return strings.Contains(errMsg, "multiple fallback services cannot be registered") - } - return true - }) - svr.Stop() + test.PanicAt(t, func() { + _ = svr.RegisterService(mocks.Service2Info(), mocks.MyServiceHandler(), WithFallbackService()) + }, func(err interface{}) bool { + if errMsg, ok := err.(string); ok { + return strings.Contains(errMsg, "multiple fallback services cannot be registered") + } + return true + }) + svr.Stop() + } - svr = NewServer() - time.AfterFunc(time.Second, func() { - err := svr.Stop() - test.Assert(t, err == nil, err) - }) + { + svr := NewServer() + time.AfterFunc(time.Second, func() { + err := svr.Stop() + test.Assert(t, err == nil, err) + }) - _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) - _ = svr.RegisterService(mocks.Service3Info(), mocks.MyServiceHandler()) - err := svr.Run() - test.Assert(t, err != nil) - test.Assert(t, err.Error() == "method name [mock] is conflicted between services but no fallback service is specified") - svr.Stop() + _ = svr.RegisterService(mocks.ServiceInfo(), mocks.MyServiceHandler()) + _ = svr.RegisterService(mocks.Service3Info(), mocks.MyServiceHandler()) + err := svr.Run() + test.Assert(t, err != nil) + test.Assert(t, err.Error() == "method name [mock] is conflicted between services but no fallback service is specified") + svr.Stop() + } } func TestRegisterServiceWithMiddleware(t *testing.T) { From b2a96650a567ca9591c17472f970d66f307ecfa0 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Wed, 22 Jan 2025 17:54:26 +0800 Subject: [PATCH 16/35] refactor(tool): rm thriftgo dep from pkg generator (#1680) --- tool/internal_pkg/generator/generator.go | 12 +++---- tool/internal_pkg/generator/generator_test.go | 19 +++-------- tool/internal_pkg/generator/type.go | 20 ++++++++++-- tool/internal_pkg/pluginmode/protoc/plugin.go | 30 ++--------------- .../pluginmode/thriftgo/convertor.go | 2 +- tool/internal_pkg/tpl/client.go | 32 +++++++++---------- tool/internal_pkg/tpl/handler.method.go | 12 +++---- tool/internal_pkg/tpl/server.go | 8 ++--- tool/internal_pkg/tpl/service.go | 22 ++++++------- 9 files changed, 67 insertions(+), 90 deletions(-) diff --git a/tool/internal_pkg/generator/generator.go b/tool/internal_pkg/generator/generator.go index d150ca7bec..4021c29591 100644 --- a/tool/internal_pkg/generator/generator.go +++ b/tool/internal_pkg/generator/generator.go @@ -569,7 +569,7 @@ func (g *generator) setImports(name string, pkg *PackageInfo) { if g.StreamX || (!m.ServerStreaming && !m.ClientStreaming) { pkg.AddImports("context") } - if g.StreamX && m.Streaming.IsStreaming { + if g.StreamX && m.IsStreaming { pkg.AddImports("github.com/cloudwego/kitex/pkg/streamx") } for _, a := range m.Args { @@ -587,7 +587,7 @@ func (g *generator) setImports(name string, pkg *PackageInfo) { // for StreamX, if there is streaming method, generate Server Interface in server.go if g.StreamX { for _, method := range pkg.AllMethods() { - if method.Streaming.IsStreaming { + if method.IsStreaming { pkg.AddImports("context") pkg.AddImports("github.com/cloudwego/kitex/pkg/streamx") } @@ -619,15 +619,15 @@ func (g *generator) setImports(name string, pkg *PackageInfo) { } // streaming imports if !g.StreamX { - if m.Streaming.IsStreaming || pkg.Codec == "protobuf" { + if m.IsStreaming || pkg.Codec == "protobuf" { // protobuf handler support both PingPong and Unary (streaming) requests pkg.AddImport("streaming", "github.com/cloudwego/kitex/pkg/streaming") } - if m.ClientStreaming || m.ServerStreaming { + if m.IsStreaming { pkg.AddImports("fmt") } } else { - if m.Streaming.IsStreaming { + if m.IsStreaming { pkg.AddImports("github.com/cloudwego/kitex/client/streamxclient") pkg.AddImports("github.com/cloudwego/kitex/client/streamxclient/streamxcallopt") pkg.AddImports("github.com/cloudwego/kitex/pkg/streamx") @@ -671,7 +671,7 @@ func needCallOpt(pkg *PackageInfo) bool { switch pkg.Codec { case "thrift": for _, m := range pkg.ServiceInfo.AllMethods() { - if !m.Streaming.IsStreaming { + if !m.IsStreaming { needCallOpt = true break } diff --git a/tool/internal_pkg/generator/generator_test.go b/tool/internal_pkg/generator/generator_test.go index 7b3d160a70..e7806c2fa0 100644 --- a/tool/internal_pkg/generator/generator_test.go +++ b/tool/internal_pkg/generator/generator_test.go @@ -19,8 +19,6 @@ import ( "testing" "time" - "github.com/cloudwego/thriftgo/generator/golang/streaming" - "github.com/cloudwego/kitex/internal/test" "github.com/cloudwego/kitex/tool/internal_pkg/util" ) @@ -202,9 +200,8 @@ func Test_needCallOpt(t *testing.T) { Codec: "thrift", ServiceInfo: &ServiceInfo{ Methods: []*MethodInfo{{ - Streaming: &streaming.Streaming{ - IsStreaming: true, - }, + IsStreaming: true, + ServerStreaming: true, }}, }, } @@ -215,16 +212,8 @@ func Test_needCallOpt(t *testing.T) { Codec: "thrift", ServiceInfo: &ServiceInfo{ Methods: []*MethodInfo{ - { - Streaming: &streaming.Streaming{ - IsStreaming: true, - }, - }, - { - Streaming: &streaming.Streaming{ - IsStreaming: false, - }, - }, + {IsStreaming: true, ServerStreaming: true}, + {}, }, }, } diff --git a/tool/internal_pkg/generator/type.go b/tool/internal_pkg/generator/type.go index 02178ded45..3ea54264d4 100644 --- a/tool/internal_pkg/generator/type.go +++ b/tool/internal_pkg/generator/type.go @@ -20,8 +20,6 @@ import ( "strings" "text/template" - "github.com/cloudwego/thriftgo/generator/golang/streaming" - "github.com/cloudwego/kitex/tool/internal_pkg/util" "github.com/cloudwego/kitex/transport" ) @@ -211,12 +209,28 @@ type MethodInfo struct { ResStructName string IsResponseNeedRedirect bool // int -> int* GenArgResultStruct bool + IsStreaming bool ClientStreaming bool ServerStreaming bool - Streaming *streaming.Streaming StreamX bool } +func (m *MethodInfo) StreamingMode() string { + if !m.IsStreaming { + return "" + } + if m.ClientStreaming { + if m.ServerStreaming { + return "bidirectional" + } + return "client" + } + if m.ServerStreaming { + return "server" + } + return "unary" +} + // Parameter . type Parameter struct { Deps []PkgInfo diff --git a/tool/internal_pkg/pluginmode/protoc/plugin.go b/tool/internal_pkg/pluginmode/protoc/plugin.go index 71700645ce..e464dcda60 100644 --- a/tool/internal_pkg/pluginmode/protoc/plugin.go +++ b/tool/internal_pkg/pluginmode/protoc/plugin.go @@ -24,7 +24,6 @@ import ( "text/template" genfastpb "github.com/cloudwego/fastpb/protoc-gen-fastpb/generator" - "github.com/cloudwego/thriftgo/generator/golang/streaming" gengo "google.golang.org/protobuf/cmd/protoc-gen-go/internal_gengo" "google.golang.org/protobuf/compiler/protogen" @@ -297,13 +296,11 @@ func (pp *protocPlugin) convertTypes(file *protogen.File) (ss []*generator.Servi ServerStreaming: m.Desc.IsStreamingServer(), } si.Methods = append(si.Methods, mi) - if !si.HasStreaming && (mi.ClientStreaming || mi.ServerStreaming) { + if mi.ClientStreaming || mi.ServerStreaming { + mi.IsStreaming = true si.HasStreaming = true } } - for _, m := range si.Methods { - BuildStreaming(m, si.HasStreaming) - } if file.Generate { si.GenerateHandler = true } @@ -348,29 +345,6 @@ func (pp *protocPlugin) convertTypes(file *protogen.File) (ss []*generator.Servi return } -// BuildStreaming builds protobuf MethodInfo.Streaming as for Thrift, to simplify codegen -func BuildStreaming(mi *generator.MethodInfo, serviceHasStreaming bool) { - s := &streaming.Streaming{ - // pb: if one method is streaming, then the service is streaming, making all methods streaming - IsStreaming: serviceHasStreaming, - } - if mi.ClientStreaming && mi.ServerStreaming { - s.Mode = streaming.StreamingBidirectional - s.BidirectionalStreaming = true - s.ClientStreaming = true - s.ServerStreaming = true - } else if mi.ClientStreaming && !mi.ServerStreaming { - s.Mode = streaming.StreamingClientSide - s.ClientStreaming = true - } else if !mi.ClientStreaming && mi.ServerStreaming { - s.Mode = streaming.StreamingServerSide - s.ServerStreaming = true - } else if serviceHasStreaming { - s.Mode = streaming.StreamingUnary // Unary APIs over HTTP2 - } - mi.Streaming = s -} - func (pp *protocPlugin) getCombineServiceName(name string, svcs []*generator.ServiceInfo) string { for _, svc := range svcs { if svc.ServiceName == name { diff --git a/tool/internal_pkg/pluginmode/thriftgo/convertor.go b/tool/internal_pkg/pluginmode/thriftgo/convertor.go index 072d08616f..d8467ee8cb 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/convertor.go +++ b/tool/internal_pkg/pluginmode/thriftgo/convertor.go @@ -455,7 +455,7 @@ func (c *converter) makeMethod(si *generator.ServiceInfo, f *golang.Function) (* Void: f.Void, ArgStructName: f.ArgType().GoName().String(), GenArgResultStruct: false, - Streaming: st, + IsStreaming: st.IsStreaming, ClientStreaming: st.ClientStreaming, ServerStreaming: st.ServerStreaming, ArgsLength: len(f.Arguments()), diff --git a/tool/internal_pkg/tpl/client.go b/tool/internal_pkg/tpl/client.go index 1d901563e2..5f9cd50db0 100644 --- a/tool/internal_pkg/tpl/client.go +++ b/tool/internal_pkg/tpl/client.go @@ -39,23 +39,23 @@ type Client interface { {{- if and (eq $.Codec "protobuf") (or .ClientStreaming .ServerStreaming)}}{{/* protobuf: generate streaming calls in Client, to keep compatibility */}} {{.Name}}(ctx context.Context {{if not .ClientStreaming}}{{range .Args}}, {{.RawName}} {{.Type}}{{end}}{{end}}, callOptions ...callopt.Option ) (stream {{.ServiceName}}_{{.RawName}}Client, err error) {{- else}} - {{- if or (eq $.Codec "protobuf") (eq .Streaming.Mode "")}} + {{- if or (eq $.Codec "protobuf") (eq .StreamingMode "")}} {{.Name}}(ctx context.Context {{range .Args}}, {{.RawName}} {{.Type}}{{end}}, callOptions ...callopt.Option ) ({{if not .Void}}r {{.Resp.Type}}, {{end}}err error) {{- end}} {{- end}} {{- /* Streamx interface for streaming method */}} -{{- if and .StreamX (eq $.Codec "thrift") .Streaming.IsStreaming}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} -{{- $clientSide := (eq .Streaming.Mode "client")}} -{{- $serverSide := (eq .Streaming.Mode "server")}} -{{- $bidiSide := (eq .Streaming.Mode "bidirectional")}} +{{- if and .StreamX (eq $.Codec "thrift") .IsStreaming}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} +{{- $clientSide := (eq .StreamingMode "client")}} +{{- $serverSide := (eq .StreamingMode "server")}} +{{- $bidiSide := (eq .StreamingMode "bidirectional")}} {{- $arg := index .Args 0}} {{.Name}}{{- if $streamingUnary}}(ctx context.Context, req {{$arg.Type}}, callOptions ...streamxcallopt.CallOption) (r {{.Resp.Type}}, err error) {{- else if $clientSide}}(ctx context.Context, callOptions ...streamxcallopt.CallOption) (context.Context, streamx.ClientStreamingClient[{{NotPtr $arg.Type}}, {{NotPtr .Resp.Type}}], error) {{- else if $serverSide}}(ctx context.Context, req {{$arg.Type}}, callOptions ...streamxcallopt.CallOption) (context.Context, streamx.ServerStreamingClient[{{NotPtr .Resp.Type}}], error) {{- else if $bidiSide}}(ctx context.Context, callOptions ...streamxcallopt.CallOption) (context.Context, streamx.BidiStreamingClient[{{NotPtr $arg.Type}}, {{NotPtr .Resp.Type}}], error) {{- end}} -{{- end}}{{- /* if and .StreamX (eq $.Codec "thrift") .Streaming.IsStreaming end */}} +{{- end}}{{- /* if and .StreamX (eq $.Codec "thrift") .IsStreaming end */}} {{- end}} } @@ -66,7 +66,7 @@ type StreamClient interface { {{- range .AllMethods}} {{- if or .ClientStreaming .ServerStreaming}} {{.Name}}(ctx context.Context {{if not .ClientStreaming}}{{range .Args}}, {{.RawName}} {{.Type}}{{end}}{{end}}, callOptions ...streamcall.Option ) (stream {{.ServiceName}}_{{.RawName}}Client, err error) -{{- else if .Streaming.Unary}} +{{- else if eq .StreamingMode "unary"}} {{.Name}}(ctx context.Context {{range .Args}}, {{.RawName}} {{.Type}}{{end}}, callOptions ...streamcall.Option ) ({{if not .Void}}r {{.Resp.Type}}, {{end}}err error) {{- end}} {{- end}} @@ -139,7 +139,7 @@ func (p *k{{$.ServiceName}}Client) {{.Name}}(ctx context.Context {{if not .Clien } {{- end}} {{- else}} -{{- if or (eq $.Codec "protobuf") (eq .Streaming.Mode "")}} +{{- if or (eq $.Codec "protobuf") (eq .StreamingMode "")}} func (p *k{{$.ServiceName}}Client) {{.Name}}(ctx context.Context {{range .Args}}, {{.RawName}} {{.Type}}{{end}}, callOptions ...callopt.Option ) ({{if not .Void}}r {{.Resp.Type}}, {{end}}err error) { ctx = client.NewCtxWithCallOptions(ctx, callOptions) return p.kClient.{{.Name}}(ctx{{range .Args}}, {{.RawName}}{{end}}) @@ -147,11 +147,11 @@ func (p *k{{$.ServiceName}}Client) {{.Name}}(ctx context.Context {{range .Args}} {{- end}} {{- end}} {{- /* Streamx interface for streaming method */}} -{{- if and .StreamX (eq $.Codec "thrift") .Streaming.IsStreaming}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} -{{- $clientSide := (eq .Streaming.Mode "client")}} -{{- $serverSide := (eq .Streaming.Mode "server")}} -{{- $bidiSide := (eq .Streaming.Mode "bidirectional")}} +{{- if and .StreamX (eq $.Codec "thrift") .IsStreaming}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} +{{- $clientSide := (eq .StreamingMode "client")}} +{{- $serverSide := (eq .StreamingMode "server")}} +{{- $bidiSide := (eq .StreamingMode "bidirectional")}} {{- $arg := index .Args 0}} func (p *k{{$.ServiceName}}Client) {{.Name}}{{- if $streamingUnary}}(ctx context.Context, req {{$arg.Type}}, callOptions ...streamxcallopt.CallOption) (r {{.Resp.Type}}, err error) { return p.kClient.{{.Name}}(ctx, req, callOptions...) @@ -163,7 +163,7 @@ func (p *k{{$.ServiceName}}Client) {{.Name}}{{- if $streamingUnary}}(ctx context return p.kClient.{{.Name}}(ctx, callOptions...) {{- end}} } -{{- end}}{{- /* if and .StreamX (eq $.Codec "thrift") .Streaming.IsStreaming end */}} +{{- end}}{{- /* if and .StreamX (eq $.Codec "thrift") .IsStreaming end */}} {{end}} {{- if not .StreamX}} @@ -207,7 +207,7 @@ func (p *k{{$.ServiceName}}StreamClient) {{.Name}}(ctx context.Context {{if not ctx = client.NewCtxWithCallOptions(ctx, streamcall.GetCallOptions(callOptions)) return p.kClient.{{.Name}}(ctx{{if not .ClientStreaming}}{{range .Args}}, {{.RawName}}{{end}}{{end}}) } -{{else if .Streaming.Unary}} +{{else if eq .StreamingMode "unary"}} func (p *k{{$.ServiceName}}StreamClient) {{.Name}}(ctx context.Context {{range .Args}}, {{.RawName}} {{.Type}}{{end}}, callOptions ...streamcall.Option ) ({{if not .Void}}r {{.Resp.Type}}, {{end}}err error) { ctx = client.NewCtxWithCallOptions(ctx, streamcall.GetCallOptions(callOptions)) return p.kClient.{{.Name}}(ctx{{range .Args}}, {{.RawName}}{{end}}) diff --git a/tool/internal_pkg/tpl/handler.method.go b/tool/internal_pkg/tpl/handler.method.go index 88a6904c63..47fd76054a 100644 --- a/tool/internal_pkg/tpl/handler.method.go +++ b/tool/internal_pkg/tpl/handler.method.go @@ -17,11 +17,11 @@ package tpl // HandlerMethodsTpl is the template for generating methods in handler.go. var HandlerMethodsTpl string = `{{define "HandlerMethod"}} {{range .AllMethods}} -{{- if and .StreamX .Streaming.IsStreaming}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} -{{- $clientSide := (eq .Streaming.Mode "client")}} -{{- $serverSide := (eq .Streaming.Mode "server")}} -{{- $bidiSide := (eq .Streaming.Mode "bidirectional")}} +{{- if and .StreamX .IsStreaming}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} +{{- $clientSide := (eq .StreamingMode "client")}} +{{- $serverSide := (eq .StreamingMode "server")}} +{{- $bidiSide := (eq .StreamingMode "bidirectional")}} {{- $arg := index .Args 0}} func (s *{{.ServiceName}}Impl) {{.Name}}{{- if $streamingUnary}}(ctx context.Context, req {{$arg.Type}}) (resp {{.Resp.Type}}, err error) { {{- else if $clientSide}}(ctx context.Context, stream streamx.ClientStreamingServer[{{NotPtr $arg.Type}}, {{NotPtr .Resp.Type}}]) (resp {{.Resp.Type}}, err error) { @@ -57,7 +57,7 @@ func (s *{{$.ServiceName}}Impl) {{.Name}}(ctx context.Context {{range .Args}}, { } {{end}}{{/* if .Void end */}} {{end}}{{/* if or .ClientStreaming .ServerStreaming end */}} -{{end}}{{/* if and .StreamX .Streaming.IsStreaming end */}} +{{end}}{{/* if and .StreamX .IsStreaming end */}} {{end}}{{/* range .AllMethods end */}} {{end}}{{/* define "HandlerMethod" */}} ` diff --git a/tool/internal_pkg/tpl/server.go b/tool/internal_pkg/tpl/server.go index 78d7cfdc9f..9cea7c250f 100644 --- a/tool/internal_pkg/tpl/server.go +++ b/tool/internal_pkg/tpl/server.go @@ -35,10 +35,10 @@ import ( {{- $serverInterfaceName = .ServiceName }} type {{.ServiceName}} interface { {{- range .AllMethods}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} -{{- $clientSide := (eq .Streaming.Mode "client")}} -{{- $serverSide := (eq .Streaming.Mode "server")}} -{{- $bidiSide := (eq .Streaming.Mode "bidirectional")}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} +{{- $clientSide := (eq .StreamingMode "client")}} +{{- $serverSide := (eq .StreamingMode "server")}} +{{- $bidiSide := (eq .StreamingMode "bidirectional")}} {{- $arg := index .Args 0}} {{.Name}}{{- if $streamingUnary}}(ctx context.Context, req {{$arg.Type}}) ({{.Resp.Type}}, error) {{- else if $clientSide}}(ctx context.Context, stream streamx.ClientStreamingServer[{{NotPtr $arg.Type}}, {{NotPtr .Resp.Type}}]) ({{.Resp.Type}}, error) diff --git a/tool/internal_pkg/tpl/service.go b/tool/internal_pkg/tpl/service.go index d574c5d8ea..e4bf26f069 100644 --- a/tool/internal_pkg/tpl/service.go +++ b/tool/internal_pkg/tpl/service.go @@ -54,11 +54,11 @@ var serviceMethods = map[string]kitex.MethodInfo{ {{- else if .ServerStreaming -}} kitex.StreamingServer {{- else if .ClientStreaming -}} kitex.StreamingClient {{- else -}} - {{- if or (eq $.Codec "protobuf") (eq .Streaming.Mode "unary") -}} kitex.StreamingUnary + {{- if or (eq $.Codec "protobuf") (eq .StreamingMode "unary") -}} kitex.StreamingUnary {{- else -}} kitex.StreamingNone {{- end -}} {{- end}}), - {{- if and .StreamX .Streaming.IsStreaming}} + {{- if and .StreamX .IsStreaming}} kitex.WithMethodExtra("streamx", "true"), {{- end}} ), @@ -146,7 +146,7 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami {{range .AllMethods}} {{- $isStreaming := or .ClientStreaming .ServerStreaming}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} {{- $unary := and (not .ServerStreaming) (not .ClientStreaming)}} {{- $clientSide := and .ClientStreaming (not .ServerStreaming)}} {{- $serverSide := and (not .ClientStreaming) .ServerStreaming}} @@ -159,7 +159,7 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami {{- else if $clientSide -}} {{- $mode = "serviceinfo.StreamingClient" }} {{- $handlerFunc = "InvokeClientStreamHandler" }} {{- else if $bidiSide -}} {{- $mode = "serviceinfo.StreamingBidirectional" }} {{- $handlerFunc = "InvokeBidiStreamHandler" }} {{- end}} -{{- if or (eq $.Codec "protobuf") ($isStreaming) (.Streaming.IsStreaming) }} +{{- if or (eq $.Codec "protobuf") ($isStreaming) (.IsStreaming) }} {{- $arg = index .Args 0}}{{/* streaming api only supports exactly one argument */}} {{- end}} @@ -205,7 +205,7 @@ func {{LowerFirst .Name}}Handler(ctx context.Context, handler interface{}, arg, return handler.({{.PkgRefName}}.{{.ServiceName}}).{{.Name}}({{if $serverSide}}req, {{end}}stream) {{- end}} {{/* $unary end */}} {{- else}} {{/* thrift logic */}} - {{- if and .StreamX .Streaming.IsStreaming}} + {{- if and .StreamX .IsStreaming}} return streamxserver.{{$handlerFunc}}[{{NotPtr $arg.Type}}, {{NotPtr .Resp.Type}}]( ctx, arg.(streamx.StreamReqArgs), result.(streamx.StreamResArgs), {{- if $streamingUnary }}func(ctx context.Context, req {{$arg.Type}}) ({{.Resp.Type}}, error) { @@ -488,11 +488,11 @@ func newServiceClient(c client.Client) *kClient { {{range .AllMethods}} {{- /* streaming logic */}} -{{- if and .StreamX .Streaming.IsStreaming}} -{{- $streamingUnary := (eq .Streaming.Mode "unary")}} -{{- $clientSide := (eq .Streaming.Mode "client")}} -{{- $serverSide := (eq .Streaming.Mode "server")}} -{{- $bidiSide := (eq .Streaming.Mode "bidirectional")}} +{{- if and .StreamX .IsStreaming}} +{{- $streamingUnary := (eq .StreamingMode "unary")}} +{{- $clientSide := (eq .StreamingMode "client")}} +{{- $serverSide := (eq .StreamingMode "server")}} +{{- $bidiSide := (eq .StreamingMode "bidirectional")}} {{- $mode := ""}} {{- if $bidiSide -}} {{- $mode = "kitex.StreamingBidirectional" }} {{- else if $serverSide -}} {{- $mode = "kitex.StreamingServer" }} @@ -595,7 +595,7 @@ func (p *kClient) {{.Name}}(ctx context.Context {{range .Args}}, {{.RawName}} {{ {{end -}} } {{- end}} -{{- end}}{{/* if and .StreamX .Streaming.IsStreaming end */}} +{{- end}}{{/* if and .StreamX .IsStreaming end */}} {{end}} {{- if .FrugalPretouch}} From d840d1ae946d37321c491592cf759d1effc3b948 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Fri, 24 Jan 2025 13:24:27 +0800 Subject: [PATCH 17/35] ci: prepare for Windows X64 hosts (#1683) --- .github/workflows/pr-check.yml | 2 +- .github/workflows/tests.yml | 28 ++++++++++++++++++++-------- 2 files changed, 21 insertions(+), 9 deletions(-) diff --git a/.github/workflows/pr-check.yml b/.github/workflows/pr-check.yml index dba739ef6f..cb4c24ff01 100644 --- a/.github/workflows/pr-check.yml +++ b/.github/workflows/pr-check.yml @@ -17,7 +17,7 @@ jobs: uses: crate-ci/typos@v1.13.14 golangci-lint: - runs-on: [ self-hosted, X64 ] + runs-on: [ Linux, X64 ] steps: - uses: actions/checkout@v4 - name: Set up Go diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 90b0c4a907..b9de72b07d 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -4,7 +4,7 @@ on: [ push, pull_request ] jobs: unit-scenario-test: - runs-on: [ self-hosted, X64 ] + runs-on: [ Linux, X64 ] steps: - uses: actions/checkout@v4 - name: Set up Go @@ -34,12 +34,26 @@ jobs: # setting benchtime=100ms is saving our time... run: go test -bench=. -benchmem -run=none ./... -benchtime=100ms - compatibility-test: + compatibility-test-x64: strategy: matrix: go: [ "1.18", "1.19", "1.20", "1.21", "1.22", "1.23" ] - os: [ X64, ARM64 ] - runs-on: ${{ matrix.os }} + runs-on: [ Linux, X64 ] + steps: + - uses: actions/checkout@v4 + - name: Set up Go + uses: actions/setup-go@v5 + with: + go-version: ${{ matrix.go }} + cache: false # don't use cache for self-hosted runners + - name: Unit Test + run: go test -race ./... + + compatibility-test-arm: + strategy: + matrix: + go: [ "1.18", "1.19", "1.20", "1.21", "1.22", "1.23" ] + runs-on: [ ARM64 ] # It's OK under Linux or macOS steps: - uses: actions/checkout@v4 - name: Set up Go @@ -80,15 +94,13 @@ jobs: bash ./codegen_run.sh windows-test: - runs-on: windows-latest - env: # Fixes https://github.com/actions/setup-go/issues/240 - GOMODCACHE: 'D:\go\pkg\mod' - GOCACHE: 'D:\go\go-build' + runs-on: [ Windows ] steps: - uses: actions/checkout@v4 - name: Set up Go uses: actions/setup-go@v5 with: go-version: stable + cache: false # don't use cache for self-hosted runners - name: Windows compatibility test run: go test -run=^$ ./... From 98e11622c83e2420e5dfa39fb90f7bfd64499f93 Mon Sep 17 00:00:00 2001 From: Li2CO3 Date: Fri, 24 Jan 2025 14:36:58 +0800 Subject: [PATCH 18/35] test: fix creating tmp dir in unit test (#1684) --- pkg/utils/yaml_test.go | 32 ++++++++------------ tool/internal_pkg/generator/template_test.go | 5 +-- 2 files changed, 16 insertions(+), 21 deletions(-) diff --git a/pkg/utils/yaml_test.go b/pkg/utils/yaml_test.go index c83a3b6b1a..9446d75d2a 100644 --- a/pkg/utils/yaml_test.go +++ b/pkg/utils/yaml_test.go @@ -18,6 +18,7 @@ package utils import ( "os" + "path/filepath" "runtime" "testing" "time" @@ -25,9 +26,9 @@ import ( "github.com/cloudwego/kitex/internal/test" ) -const ( - TestYamlFile = "/tmp/test.yaml" -) +func getTestYamlFile(t *testing.T) string { + return filepath.Join(t.TempDir(), "test.yaml") +} var cfg *YamlConfig @@ -78,33 +79,26 @@ MockFloat64: 123456.789` } } -func deleteTestYamlFile(t *testing.T, path string) { - if err := os.Remove(path); err != nil { - t.Errorf("Remove file: %s error, err: %s", path, err.Error()) - } -} - func TestMain(m *testing.M) { if runtime.GOOS == "windows" { return } t := &testing.T{} - createTestYamlFile(t, TestYamlFile) - cfg, _ = ReadYamlConfigFile(TestYamlFile) + testYamlFile := getTestYamlFile(t) + + createTestYamlFile(t, testYamlFile) + + cfg, _ = ReadYamlConfigFile(testYamlFile) - exit := m.Run() - deleteTestYamlFile(t, TestYamlFile) - os.Exit(exit) + os.Exit(m.Run()) } func Test_ReadYamlConfigFile(t *testing.T) { - createTestYamlFile(t, TestYamlFile) - defer func() { - deleteTestYamlFile(t, TestYamlFile) - }() + testYamlFile := getTestYamlFile(t) + createTestYamlFile(t, testYamlFile) - cfg, err := ReadYamlConfigFile(TestYamlFile) + cfg, err := ReadYamlConfigFile(testYamlFile) test.Assert(t, err == nil) addr, ok := cfg.GetString("Address") test.Assert(t, ok && addr == ":8888") diff --git a/tool/internal_pkg/generator/template_test.go b/tool/internal_pkg/generator/template_test.go index 79d464006f..4b3b5520e5 100644 --- a/tool/internal_pkg/generator/template_test.go +++ b/tool/internal_pkg/generator/template_test.go @@ -15,6 +15,7 @@ package generator import ( + "path/filepath" "testing" "github.com/cloudwego/kitex/internal/test" @@ -22,7 +23,7 @@ import ( func TestNilSafe(t *testing.T) { var q *TemplateExtension - fn := "/tmp/kitex-template-nil.json" + fn := filepath.Join(t.TempDir(), "kitex-template-nil.json") err := q.ToJSONFile(fn) test.Assert(t, err == nil, err) @@ -56,7 +57,7 @@ func TestMarshal(t *testing.T) { }, } - fn := "/tmp/kitex-template.json" + fn := filepath.Join(t.TempDir(), "kitex-template.json") err := p.ToJSONFile(fn) test.Assert(t, err == nil, err) From 047444c8e96442548b7351f02b5bf4f0b8bd1b7b Mon Sep 17 00:00:00 2001 From: Jayant Date: Fri, 24 Jan 2025 15:27:55 +0800 Subject: [PATCH 19/35] feat: set crrst flag on response header to ensure kitex client won't reuse bad connections (#1653) --- pkg/remote/trans/default_server_handler.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/pkg/remote/trans/default_server_handler.go b/pkg/remote/trans/default_server_handler.go index 0b557428e9..7a7c681804 100644 --- a/pkg/remote/trans/default_server_handler.go +++ b/pkg/remote/trans/default_server_handler.go @@ -187,7 +187,7 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) recvMsg.SetPayloadCodec(t.opt.PayloadCodec) ctx, err = t.transPipe.Read(ctx, conn, recvMsg) if err != nil { - t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, true) + t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, true, true) // t.OnError(ctx, err, conn) will be executed at outer function when transServer close the conn return err } @@ -203,7 +203,7 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) var methodInfo serviceinfo.MethodInfo if methodInfo, err = GetMethodInfo(ri, svcInfo); err != nil { // it won't be error, because the method has been checked in decode, err check here just do defensive inspection - t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, true) + t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, true, true) // for proxy case, need read actual remoteAddr, error print must exec after writeErrorReplyIfNeeded, // t.OnError(ctx, err, conn) will be executed at outer function when transServer close the conn return err @@ -219,7 +219,7 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) // error cannot be wrapped to print here, so it must exec before NewTransError t.OnError(ctx, err, conn) err = remote.NewTransError(remote.InternalError, err) - if closeConn := t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, false); closeConn { + if closeConn := t.writeErrorReplyIfNeeded(ctx, recvMsg, conn, err, ri, false, false); closeConn { return err } // connection don't need to be closed when the error is return by the server handler @@ -288,7 +288,7 @@ func (t *svrTransHandler) SetPipeline(p *remote.TransPipeline) { } func (t *svrTransHandler) writeErrorReplyIfNeeded( - ctx context.Context, recvMsg remote.Message, conn net.Conn, err error, ri rpcinfo.RPCInfo, doOnMessage bool, + ctx context.Context, recvMsg remote.Message, conn net.Conn, err error, ri rpcinfo.RPCInfo, doOnMessage, connReset bool, ) (shouldCloseConn bool) { if cn, ok := conn.(remote.IsActive); ok && !cn.IsActive() { // conn is closed, no need reply @@ -313,6 +313,13 @@ func (t *svrTransHandler) writeErrorReplyIfNeeded( // if error happen before normal OnMessage, exec it to transfer header trans info into rpcinfo t.transPipe.OnMessage(ctx, recvMsg, errMsg) } + if connReset { + // if connection needs to be closed, set ConnResetTag to response header + // to ensure the client won't reuse the connection. + if ei := rpcinfo.AsTaggable(ri.To()); ei != nil { + ei.SetTag(rpcinfo.ConnResetTag, "1") + } + } ctx, err = t.transPipe.Write(ctx, conn, errMsg) if err != nil { klog.CtxErrorf(ctx, "KITEX: write error reply failed, remote=%s, error=%s", conn.RemoteAddr(), err.Error()) From 9c630f8db7ea3c73a6ba940e36db9162ba731715 Mon Sep 17 00:00:00 2001 From: Marina Sakai <118230951+Marina-Sakai@users.noreply.github.com> Date: Mon, 10 Feb 2025 17:57:15 +0800 Subject: [PATCH 20/35] fix(generic): fix codec to be updated even if there is an idl update (#1666) --- .../genericclient/generic_stream_service.go | 19 +++++---- pkg/generic/generic.go | 8 ++-- pkg/generic/generic_service.go | 40 ++++++++++++++++--- pkg/generic/httppbthrift_codec.go | 30 +++++++------- pkg/generic/httppbthrift_codec_test.go | 4 +- pkg/generic/httpthrift_codec.go | 32 +++++++++++---- pkg/generic/httpthrift_codec_test.go | 8 +--- pkg/generic/jsonpb_codec.go | 15 +++---- pkg/generic/jsonpb_codec_test.go | 4 +- pkg/generic/jsonthrift_codec.go | 33 +++++++++++---- pkg/generic/jsonthrift_codec_test.go | 8 +--- pkg/generic/mapthrift_codec.go | 40 ++++++++++++++----- pkg/generic/mapthrift_codec_test.go | 8 +--- pkg/generic/pbidl_provider.go | 4 ++ pkg/generic/thriftidl_provider.go | 6 ++- 15 files changed, 167 insertions(+), 92 deletions(-) diff --git a/client/genericclient/generic_stream_service.go b/client/genericclient/generic_stream_service.go index 44698ea26a..bd92d1e383 100644 --- a/client/genericclient/generic_stream_service.go +++ b/client/genericclient/generic_stream_service.go @@ -26,8 +26,7 @@ func StreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { } func newClientStreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { - readerWriter := g.MessageReaderWriter() - if readerWriter == nil { + if g.PayloadCodec() != nil { // TODO: support grpc binary generic panic("binary generic streaming is not supported") } @@ -37,12 +36,12 @@ func newClientStreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { nil, func() interface{} { args := &generic.Args{} - args.SetCodec(readerWriter) + args.SetCodec(g.MessageReaderWriter()) return args }, func() interface{} { result := &generic.Result{} - result.SetCodec(readerWriter) + result.SetCodec(g.MessageReaderWriter()) return result }, false, @@ -52,12 +51,12 @@ func newClientStreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { nil, func() interface{} { args := &generic.Args{} - args.SetCodec(readerWriter) + args.SetCodec(g.MessageReaderWriter()) return args }, func() interface{} { result := &generic.Result{} - result.SetCodec(readerWriter) + result.SetCodec(g.MessageReaderWriter()) return result }, false, @@ -67,12 +66,12 @@ func newClientStreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { nil, func() interface{} { args := &generic.Args{} - args.SetCodec(readerWriter) + args.SetCodec(g.MessageReaderWriter()) return args }, func() interface{} { result := &generic.Result{} - result.SetCodec(readerWriter) + result.SetCodec(g.MessageReaderWriter()) return result }, false, @@ -82,12 +81,12 @@ func newClientStreamingServiceInfo(g generic.Generic) *serviceinfo.ServiceInfo { nil, func() interface{} { args := &generic.Args{} - args.SetCodec(readerWriter) + args.SetCodec(g.MessageReaderWriter()) return args }, func() interface{} { result := &generic.Result{} - result.SetCodec(readerWriter) + result.SetCodec(g.MessageReaderWriter()) return result }, false, diff --git a/pkg/generic/generic.go b/pkg/generic/generic.go index e69757b856..5e99cde6c4 100644 --- a/pkg/generic/generic.go +++ b/pkg/generic/generic.go @@ -142,6 +142,7 @@ func SetBinaryWithBase64(g Generic, enable bool) error { c.codec.convOpts.NoBase64Binary = !enable c.codec.convOptsWithThriftBase.NoBase64Binary = !enable } + return c.codec.updateMessageReaderWriter() case *jsonThriftGeneric: if c.codec == nil { return fmt.Errorf("empty codec for %#v", c) @@ -152,15 +153,16 @@ func SetBinaryWithBase64(g Generic, enable bool) error { c.codec.convOptsWithThriftBase.NoBase64Binary = !enable c.codec.convOptsWithException.NoBase64Binary = !enable } + return c.codec.updateMessageReaderWriter() case *mapThriftGeneric: if c.codec == nil { return fmt.Errorf("empty codec for %#v", c) } c.codec.binaryWithBase64 = enable + return c.codec.updateMessageReaderWriter() default: return fmt.Errorf("Base64Binary is unavailable for %#v", g) } - return nil } // SetBinaryWithByteSlice enable/disable returning []byte for binary field. @@ -171,10 +173,10 @@ func SetBinaryWithByteSlice(g Generic, enable bool) error { return fmt.Errorf("empty codec for %#v", c) } c.codec.binaryWithByteSlice = enable + return c.codec.updateMessageReaderWriter() default: return fmt.Errorf("returning []byte for binary fields is unavailable for %#v", g) } - return nil } // SetFieldsForEmptyStructMode is a enum for EnableSetFieldsForEmptyStruct() @@ -202,10 +204,10 @@ func EnableSetFieldsForEmptyStruct(g Generic, mode SetFieldsForEmptyStructMode) return fmt.Errorf("empty codec for %#v", c) } c.codec.setFieldsForEmptyStruct = uint8(mode) + return c.codec.updateMessageReaderWriter() default: return fmt.Errorf("SetFieldsForEmptyStruct only supports map-generic at present") } - return nil } var thriftCodec = thrift.NewThriftCodec() diff --git a/pkg/generic/generic_service.go b/pkg/generic/generic_service.go index 5bff8c6f1c..0b23511451 100644 --- a/pkg/generic/generic_service.go +++ b/pkg/generic/generic_service.go @@ -32,7 +32,7 @@ type Service interface { // ServiceInfoWithGeneric create a generic ServiceInfo func ServiceInfoWithGeneric(g Generic) *serviceinfo.ServiceInfo { isCombinedServices := getIsCombinedServices(g) - return newServiceInfo(g.PayloadCodecType(), g.MessageReaderWriter(), g.IDLServiceName(), isCombinedServices) + return newServiceInfo(g, isCombinedServices) } func getIsCombinedServices(g Generic) bool { @@ -44,16 +44,16 @@ func getIsCombinedServices(g Generic) bool { return false } -func newServiceInfo(pcType serviceinfo.PayloadCodec, messageReaderWriter interface{}, serviceName string, isCombinedServices bool) *serviceinfo.ServiceInfo { +func newServiceInfo(g Generic, isCombinedServices bool) *serviceinfo.ServiceInfo { handlerType := (*Service)(nil) - methods, svcName := GetMethodInfo(messageReaderWriter, serviceName) + methods, svcName := getMethodInfo(g, g.IDLServiceName()) svcInfo := &serviceinfo.ServiceInfo{ ServiceName: svcName, HandlerType: handlerType, Methods: methods, - PayloadCodec: pcType, + PayloadCodec: g.PayloadCodecType(), Extra: make(map[string]interface{}), } svcInfo.Extra["generic"] = true @@ -63,7 +63,37 @@ func newServiceInfo(pcType serviceinfo.PayloadCodec, messageReaderWriter interfa return svcInfo } -// GetMethodInfo is only used in kitex, please DON'T USE IT. This method may be removed in the future +func getMethodInfo(g Generic, serviceName string) (methods map[string]serviceinfo.MethodInfo, svcName string) { + if g.PayloadCodec() != nil { + // note: binary generic cannot be used with multi-service feature + svcName = serviceinfo.GenericService + methods = map[string]serviceinfo.MethodInfo{ + serviceinfo.GenericMethod: serviceinfo.NewMethodInfo(callHandler, newGenericServiceCallArgs, newGenericServiceCallResult, false), + } + } else { + svcName = serviceName + methods = map[string]serviceinfo.MethodInfo{ + serviceinfo.GenericMethod: serviceinfo.NewMethodInfo( + callHandler, + func() interface{} { + args := &Args{} + args.SetCodec(g.MessageReaderWriter()) + return args + }, + func() interface{} { + result := &Result{} + result.SetCodec(g.MessageReaderWriter()) + return result + }, + false, + ), + } + } + return +} + +// GetMethodInfo is only used in kitex, please DON'T USE IT. +// DEPRECATED: this method is no longer used. This method will be removed in the future func GetMethodInfo(messageReaderWriter interface{}, serviceName string) (methods map[string]serviceinfo.MethodInfo, svcName string) { if messageReaderWriter == nil { // note: binary generic cannot be used with multi-service feature diff --git a/pkg/generic/httppbthrift_codec.go b/pkg/generic/httppbthrift_codec.go index 0e546eb1c4..94f4f0a4d9 100644 --- a/pkg/generic/httppbthrift_codec.go +++ b/pkg/generic/httppbthrift_codec.go @@ -19,6 +19,7 @@ package generic import ( "context" "errors" + "fmt" "io" "net/http" "strings" @@ -37,12 +38,13 @@ import ( var _ Closer = &httpPbThriftCodec{} type httpPbThriftCodec struct { - svcDsc atomic.Value // *idl - pbSvcDsc atomic.Value // *pbIdl - provider DescriptorProvider - pbProvider PbDescriptorProvider - svcName string - extra map[string]string + svcDsc atomic.Value // *idl + pbSvcDsc atomic.Value // *pbIdl + provider DescriptorProvider + pbProvider PbDescriptorProvider + svcName string + extra map[string]string + readerWriter atomic.Value // *thrift.HTTPPbReaderWriter } func newHTTPPbThriftCodec(p DescriptorProvider, pbp PbDescriptorProvider) *httpPbThriftCodec { @@ -57,6 +59,7 @@ func newHTTPPbThriftCodec(p DescriptorProvider, pbp PbDescriptorProvider) *httpP c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) c.pbSvcDsc.Store(pbSvc) + c.readerWriter.Store(thrift.NewHTTPPbReaderWriter(svc, pbSvc)) go c.update() return c } @@ -77,6 +80,7 @@ func (c *httpPbThriftCodec) update() { c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) c.pbSvcDsc.Store(pbSvc) + c.readerWriter.Store(thrift.NewHTTPPbReaderWriter(svc, pbSvc)) } } @@ -105,16 +109,12 @@ func (c *httpPbThriftCodec) getMethod(req interface{}) (*Method, error) { } func (c *httpPbThriftCodec) getMessageReaderWriter() interface{} { - svcDsc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) - if !ok { - return errors.New("get parser ServiceDescriptor failed") - } - pbSvcDsc, ok := c.pbSvcDsc.Load().(*desc.ServiceDescriptor) - if !ok { - return errors.New("get parser PbServiceDescriptor failed") + v := c.readerWriter.Load() + if rw, ok := v.(*thrift.HTTPPbReaderWriter); !ok { + panic(fmt.Sprintf("get readerWriter failed: expected *thrift.HTTPPbReaderWriter, got %T", v)) + } else { + return rw } - - return thrift.NewHTTPPbReaderWriter(svcDsc, pbSvcDsc) } func (c *httpPbThriftCodec) Name() string { diff --git a/pkg/generic/httppbthrift_codec_test.go b/pkg/generic/httppbthrift_codec_test.go index d1f669e0ea..062eb59283 100644 --- a/pkg/generic/httppbthrift_codec_test.go +++ b/pkg/generic/httppbthrift_codec_test.go @@ -71,8 +71,6 @@ func TestHTTPPbThriftCodec(t *testing.T) { test.Assert(t, htc.extra[CombineServiceKey] == "false") rw := htc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.HTTPPbReaderWriter) test.Assert(t, ok) } diff --git a/pkg/generic/httpthrift_codec.go b/pkg/generic/httpthrift_codec.go index 4644f37449..c13126ef15 100644 --- a/pkg/generic/httpthrift_codec.go +++ b/pkg/generic/httpthrift_codec.go @@ -19,6 +19,7 @@ package generic import ( "context" "errors" + "fmt" "io" "net/http" "sync/atomic" @@ -50,6 +51,7 @@ type httpThriftCodec struct { useRawBodyForHTTPResp bool svcName string extra map[string]string + readerWriter atomic.Value // *thrift.HTTPReaderWriter } func newHTTPThriftCodec(p DescriptorProvider, opts *Options) *httpThriftCodec { @@ -73,6 +75,7 @@ func newHTTPThriftCodec(p DescriptorProvider, opts *Options) *httpThriftCodec { } c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) go c.update() return c } @@ -86,9 +89,26 @@ func (c *httpThriftCodec) update() { c.svcName = svc.Name c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) } } +func (c *httpThriftCodec) updateMessageReaderWriter() (err error) { + svc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) + if !ok { + return errors.New("get parser ServiceDescriptor failed") + } + c.configureMessageReaderWriter(svc) + return nil +} + +func (c *httpThriftCodec) configureMessageReaderWriter(svc *descriptor.ServiceDescriptor) { + rw := thrift.NewHTTPReaderWriter(svc) + c.configureHTTPRequestWriter(rw.WriteHTTPRequest) + c.configureHTTPResponseReader(rw.ReadHTTPResponse) + c.readerWriter.Store(rw) +} + func (c *httpThriftCodec) setCombinedServices(isCombinedServices bool) { if isCombinedServices { c.extra[CombineServiceKey] = "true" @@ -98,14 +118,12 @@ func (c *httpThriftCodec) setCombinedServices(isCombinedServices bool) { } func (c *httpThriftCodec) getMessageReaderWriter() interface{} { - svcDsc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) - if !ok { - return errors.New("get parser ServiceDescriptor failed") + v := c.readerWriter.Load() + if rw, ok := v.(*thrift.HTTPReaderWriter); !ok { + panic(fmt.Sprintf("get readerWriter failed: expected *thrift.HTTPReaderWriter, got %T", v)) + } else { + return rw } - rw := thrift.NewHTTPReaderWriter(svcDsc) - c.configureHTTPRequestWriter(rw.WriteHTTPRequest) - c.configureHTTPResponseReader(rw.ReadHTTPResponse) - return rw } func (c *httpThriftCodec) configureHTTPRequestWriter(writer *thrift.WriteHTTPRequest) { diff --git a/pkg/generic/httpthrift_codec_test.go b/pkg/generic/httpthrift_codec_test.go index 4457c19891..edc127cc91 100644 --- a/pkg/generic/httpthrift_codec_test.go +++ b/pkg/generic/httpthrift_codec_test.go @@ -72,9 +72,7 @@ func TestHttpThriftCodec(t *testing.T) { test.Assert(t, !ok) rw = htc.getMessageReaderWriter() - _, ok = rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok = rw.(*thrift.HTTPReaderWriter) test.Assert(t, ok) } @@ -105,9 +103,7 @@ func TestHttpThriftCodecWithDynamicGo(t *testing.T) { test.Assert(t, htc.extra[CombineServiceKey] == "false") rw := htc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.HTTPReaderWriter) test.Assert(t, ok) } diff --git a/pkg/generic/jsonpb_codec.go b/pkg/generic/jsonpb_codec.go index fcae93ae9e..33d2cd65c6 100644 --- a/pkg/generic/jsonpb_codec.go +++ b/pkg/generic/jsonpb_codec.go @@ -18,7 +18,6 @@ package generic import ( "context" - "errors" "fmt" "sync/atomic" @@ -42,6 +41,7 @@ type jsonPbCodec struct { dynamicgoEnabled bool // currently set to true by default svcName string extra map[string]string + readerWriter atomic.Value // *proto.JSONReaderWriter } func newJsonPbCodec(p PbDescriptorProviderDynamicGo, opts *Options) *jsonPbCodec { @@ -57,8 +57,8 @@ func newJsonPbCodec(p PbDescriptorProviderDynamicGo, opts *Options) *jsonPbCodec c.convOpts = convOpts c.setCombinedServices(svc.IsCombinedServices()) c.setPackageName(svc.PackageName()) - c.svcDsc.Store(svc) + c.readerWriter.Store(proto.NewJsonReaderWriter(svc, &convOpts)) go c.update() return c } @@ -72,6 +72,7 @@ func (c *jsonPbCodec) update() { c.svcName = svc.Name() c.setCombinedServices(svc.IsCombinedServices()) c.svcDsc.Store(svc) + c.readerWriter.Store(proto.NewJsonReaderWriter(svc, &c.convOpts)) } } @@ -88,12 +89,12 @@ func (c *jsonPbCodec) setPackageName(pkg string) { } func (c *jsonPbCodec) getMessageReaderWriter() interface{} { - pbSvc, ok := c.svcDsc.Load().(*dproto.ServiceDescriptor) - if !ok { - return errors.New("get parser dynamicgo ServiceDescriptor failed") + v := c.readerWriter.Load() + if rw, ok := v.(*proto.JSONReaderWriter); !ok { + panic(fmt.Sprintf("get readerWriter failed: expected *proto.JSONReaderWriter, got %T", v)) + } else { + return rw } - - return proto.NewJsonReaderWriter(pbSvc, &c.convOpts) } func (c *jsonPbCodec) getMethod(req interface{}, method string) (*Method, error) { diff --git a/pkg/generic/jsonpb_codec_test.go b/pkg/generic/jsonpb_codec_test.go index 43a84e84a5..2baffa1364 100644 --- a/pkg/generic/jsonpb_codec_test.go +++ b/pkg/generic/jsonpb_codec_test.go @@ -56,9 +56,7 @@ func TestJsonPbCodec(t *testing.T) { test.Assert(t, jpc.extra[packageNameKey] == "test") rw := jpc.getMessageReaderWriter() - _, ok := rw.(gproto.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(gproto.MessageReader) + _, ok := rw.(*gproto.JSONReaderWriter) test.Assert(t, ok) p, err = NewPbFileProviderWithDynamicGo(testIDLPath, context.Background(), opts) diff --git a/pkg/generic/jsonthrift_codec.go b/pkg/generic/jsonthrift_codec.go index f6455971db..d65ba49a22 100644 --- a/pkg/generic/jsonthrift_codec.go +++ b/pkg/generic/jsonthrift_codec.go @@ -19,6 +19,7 @@ package generic import ( "context" "errors" + "fmt" "sync/atomic" "github.com/cloudwego/dynamicgo/conv" @@ -43,6 +44,7 @@ type jsonThriftCodec struct { convOptsWithException conv.Options // used for dynamicgo conversion with ConvertException turned on svcName string extra map[string]string + readerWriter atomic.Value // *thrift.JSONReaderWriter } func newJsonThriftCodec(p DescriptorProvider, opts *Options) *jsonThriftCodec { @@ -72,6 +74,7 @@ func newJsonThriftCodec(p DescriptorProvider, opts *Options) *jsonThriftCodec { c.convOptsWithException = convOptsWithException } c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) go c.update() return c } @@ -85,9 +88,26 @@ func (c *jsonThriftCodec) update() { c.svcName = svc.Name c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) } } +func (c *jsonThriftCodec) updateMessageReaderWriter() (err error) { + svc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) + if !ok { + return errors.New("get parser ServiceDescriptor failed") + } + c.configureMessageReaderWriter(svc) + return nil +} + +func (c *jsonThriftCodec) configureMessageReaderWriter(svc *descriptor.ServiceDescriptor) { + rw := thrift.NewJsonReaderWriter(svc) + c.configureJSONWriter(rw.WriteJSON) + c.configureJSONReader(rw.ReadJSON) + c.readerWriter.Store(rw) +} + func (c *jsonThriftCodec) setCombinedServices(isCombinedServices bool) { if isCombinedServices { c.extra[CombineServiceKey] = "true" @@ -97,15 +117,12 @@ func (c *jsonThriftCodec) setCombinedServices(isCombinedServices bool) { } func (c *jsonThriftCodec) getMessageReaderWriter() interface{} { - svcDsc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) - if !ok { - return errors.New("get parser ServiceDescriptor failed") + v := c.readerWriter.Load() + if rw, ok := v.(*thrift.JSONReaderWriter); !ok { + panic(fmt.Sprintf("get readerWriter failed: expected *thrift.JSONReaderWriter, got %T", v)) + } else { + return rw } - - rw := thrift.NewJsonReaderWriter(svcDsc) - c.configureJSONWriter(rw.WriteJSON) - c.configureJSONReader(rw.ReadJSON) - return rw } func (c *jsonThriftCodec) configureJSONWriter(writer *thrift.WriteJSON) { diff --git a/pkg/generic/jsonthrift_codec_test.go b/pkg/generic/jsonthrift_codec_test.go index f287afe0ac..69132b5b00 100644 --- a/pkg/generic/jsonthrift_codec_test.go +++ b/pkg/generic/jsonthrift_codec_test.go @@ -46,9 +46,7 @@ func TestJsonThriftCodec(t *testing.T) { test.Assert(t, jtc.extra[CombineServiceKey] == "false") rw := jtc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.JSONReaderWriter) test.Assert(t, ok) } @@ -76,9 +74,7 @@ func TestJsonThriftCodecWithDynamicGo(t *testing.T) { test.Assert(t, jtc.extra[CombineServiceKey] == "false") rw := jtc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.JSONReaderWriter) test.Assert(t, ok) } diff --git a/pkg/generic/mapthrift_codec.go b/pkg/generic/mapthrift_codec.go index c35a6474a4..cf300b65d8 100644 --- a/pkg/generic/mapthrift_codec.go +++ b/pkg/generic/mapthrift_codec.go @@ -19,6 +19,7 @@ package generic import ( "context" "errors" + "fmt" "sync/atomic" "github.com/cloudwego/kitex/pkg/generic/descriptor" @@ -39,6 +40,7 @@ type mapThriftCodec struct { setFieldsForEmptyStruct uint8 svcName string extra map[string]string + readerWriter atomic.Value // *thrift.StructReaderWriter } func newMapThriftCodec(p DescriptorProvider) *mapThriftCodec { @@ -52,6 +54,7 @@ func newMapThriftCodec(p DescriptorProvider) *mapThriftCodec { } c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) go c.update() return c } @@ -71,9 +74,31 @@ func (c *mapThriftCodec) update() { c.svcName = svc.Name c.setCombinedServices(svc.IsCombinedServices) c.svcDsc.Store(svc) + c.configureMessageReaderWriter(svc) } } +func (c *mapThriftCodec) updateMessageReaderWriter() (err error) { + svc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) + if !ok { + return errors.New("get parser ServiceDescriptor failed") + } + c.configureMessageReaderWriter(svc) + return nil +} + +func (c *mapThriftCodec) configureMessageReaderWriter(svc *descriptor.ServiceDescriptor) { + var rw *thrift.StructReaderWriter + if c.forJSON { + rw = thrift.NewStructReaderWriterForJSON(svc) + } else { + rw = thrift.NewStructReaderWriter(svc) + } + c.configureStructWriter(rw.WriteStruct) + c.configureStructReader(rw.ReadStruct) + c.readerWriter.Store(rw) +} + func (c *mapThriftCodec) setCombinedServices(isCombinedServices bool) { if isCombinedServices { c.extra[CombineServiceKey] = "true" @@ -83,19 +108,12 @@ func (c *mapThriftCodec) setCombinedServices(isCombinedServices bool) { } func (c *mapThriftCodec) getMessageReaderWriter() interface{} { - svcDsc, ok := c.svcDsc.Load().(*descriptor.ServiceDescriptor) - if !ok { - return errors.New("get parser ServiceDescriptor failed") - } - var rw *thrift.StructReaderWriter - if c.forJSON { - rw = thrift.NewStructReaderWriterForJSON(svcDsc) + v := c.readerWriter.Load() + if rw, ok := v.(*thrift.StructReaderWriter); !ok { + panic(fmt.Sprintf("get readerWriter failed: expected *thrift.StructReaderWriter, got %T", v)) } else { - rw = thrift.NewStructReaderWriter(svcDsc) + return rw } - c.configureStructWriter(rw.WriteStruct) - c.configureStructReader(rw.ReadStruct) - return rw } func (c *mapThriftCodec) configureStructWriter(writer *thrift.WriteStruct) { diff --git a/pkg/generic/mapthrift_codec_test.go b/pkg/generic/mapthrift_codec_test.go index 7be2bd67ef..92d103d3b6 100644 --- a/pkg/generic/mapthrift_codec_test.go +++ b/pkg/generic/mapthrift_codec_test.go @@ -59,9 +59,7 @@ func TestMapThriftCodecSelfRef(t *testing.T) { test.Assert(t, mtc.svcName == "Mock") rw := mtc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.StructReaderWriter) test.Assert(t, ok) } @@ -79,8 +77,6 @@ func TestMapThriftCodecForJSON(t *testing.T) { test.Assert(t, mtc.svcName == "Mock") rw := mtc.getMessageReaderWriter() - _, ok := rw.(thrift.MessageWriter) - test.Assert(t, ok) - _, ok = rw.(thrift.MessageReader) + _, ok := rw.(*thrift.StructReaderWriter) test.Assert(t, ok) } diff --git a/pkg/generic/pbidl_provider.go b/pkg/generic/pbidl_provider.go index 32651c8007..4521d3b518 100644 --- a/pkg/generic/pbidl_provider.go +++ b/pkg/generic/pbidl_provider.go @@ -56,6 +56,8 @@ func NewPbContentProvider(main string, includes map[string]string) (PbDescriptor return p, nil } +// UpdateIDL updates idl +// NOTE: Since an IDL update is asynchronous, it may not be applied immediately, potentially causing a temporary data inconsistency. func (p *PbContentProvider) UpdateIDL(main string, includes map[string]string) error { sd, err := parseProto(main, includes) if err != nil { @@ -140,6 +142,8 @@ func NewPbContentProviderWithDynamicGo(ctx context.Context, options dproto.Optio return p, nil } +// UpdateIDL updates idl +// NOTE: Since an IDL update is asynchronous, it may not be applied immediately, potentially causing a temporary data inconsistency. func (p *PbFileProviderWithDynamicGo) UpdateIDL(ctx context.Context, options dproto.Options, mainPath, mainContent string, includes map[string]string) error { sd, err := options.NewDesccriptorFromContent(ctx, mainPath, mainContent, includes) if err != nil { diff --git a/pkg/generic/thriftidl_provider.go b/pkg/generic/thriftidl_provider.go index fc89e333ba..82cd96d461 100644 --- a/pkg/generic/thriftidl_provider.go +++ b/pkg/generic/thriftidl_provider.go @@ -205,7 +205,8 @@ func NewThriftContentProviderWithDynamicGo(main string, includes map[string]stri return p, nil } -// UpdateIDL ... +// UpdateIDL updates idl +// NOTE: Since an IDL update is asynchronous, it may not be applied immediately, potentially causing a temporary data inconsistency. func (p *ThriftContentProvider) UpdateIDL(main string, includes map[string]string) error { var svc *descriptor.ServiceDescriptor tree, err := ParseContent(defaultMainIDLPath, main, includes, false) @@ -388,7 +389,8 @@ func NewThriftContentWithAbsIncludePathProviderWithDynamicGo(mainIDLPath string, return p, nil } -// UpdateIDL update idl by given args +// UpdateIDL updates idl +// NOTE: Since an IDL update is asynchronous, it may not be applied immediately, potentially causing a temporary data inconsistency. func (p *ThriftContentWithAbsIncludePathProvider) UpdateIDL(mainIDLPath string, includes map[string]string) error { mainIDLContent, ok := includes[mainIDLPath] if !ok { From 0fed92cb8753546b107cefa92a329e78c501065c Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Tue, 11 Feb 2025 15:19:52 +0800 Subject: [PATCH 21/35] test(server): fix address already in use (#1691) --- server/server_test.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/server/server_test.go b/server/server_test.go index 1b479027cf..7a6b66c94c 100644 --- a/server/server_test.go +++ b/server/server_test.go @@ -742,9 +742,9 @@ func TestServerBoundHandler(t *testing.T) { } func TestInvokeHandlerWithContextBackup(t *testing.T) { - testInvokeHandlerWithSession(t, true, "localhost:8888") + testInvokeHandlerWithSession(t, true, "localhost:0") os.Setenv(localsession.SESSION_CONFIG_KEY, "true,100,1h") - testInvokeHandlerWithSession(t, false, "localhost:8889") + testInvokeHandlerWithSession(t, false, "localhost:0") } func testInvokeHandlerWithSession(t *testing.T, fail bool, ad string) { @@ -891,7 +891,7 @@ func TestInvokeHandlerExec(t *testing.T) { }, CreateListenerFunc: func(addr net.Addr) (net.Listener, error) { var err error - ln, err = net.Listen("tcp", "localhost:8888") + ln, err = net.Listen("tcp", "localhost:0") return ln, err }, } @@ -954,7 +954,7 @@ func TestInvokeHandlerPanic(t *testing.T) { }, CreateListenerFunc: func(addr net.Addr) (net.Listener, error) { var err error - ln, err = net.Listen("tcp", "localhost:8888") + ln, err = net.Listen("tcp", "localhost:0") return ln, err }, } @@ -1093,7 +1093,7 @@ func TestRegisterServiceWithMiddleware(t *testing.T) { }, CreateListenerFunc: func(addr net.Addr) (net.Listener, error) { var err error - ln, err = net.Listen("tcp", "localhost:8888") + ln, err = net.Listen("tcp", "localhost:0") return ln, err }, } From 56a1e368efdf9863976903d7840c8a194b36f79b Mon Sep 17 00:00:00 2001 From: Li2CO3 Date: Tue, 11 Feb 2025 20:34:12 +0800 Subject: [PATCH 22/35] fix(tool): some bugfix for kitex tool (#1692) --- tool/cmd/kitex/args/args.go | 118 ++------------------ tool/cmd/kitex/args/args_test.go | 31 ----- tool/cmd/kitex/args/version_requirements.go | 17 --- tool/cmd/kitex/main.go | 3 - 4 files changed, 11 insertions(+), 158 deletions(-) delete mode 100644 tool/cmd/kitex/args/version_requirements.go diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index 9a0aa7c3c3..c4ad137549 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -15,7 +15,6 @@ package args import ( - "context" "errors" "flag" "fmt" @@ -24,10 +23,7 @@ import ( "os/exec" "path" "path/filepath" - "regexp" - "strconv" "strings" - "time" "github.com/cloudwego/kitex/tool/internal_pkg/generator" "github.com/cloudwego/kitex/tool/internal_pkg/log" @@ -128,10 +124,8 @@ func (a *Arguments) buildFlags(version string) *flag.FlagSet { "Specify a protocol for codec") f.BoolVar(&a.NoDependencyCheck, "no-dependency-check", false, "Skip dependency checking.") - f.BoolVar(&a.Rapid, "rapid", false, - "Try some experimental features to generate code faster.") - f.BoolVar(&a.LocalThriftgo, "local_thriftgo", false, - "Use local thriftgo exec instead of kitex embedded thriftgo.") + f.BoolVar(&a.LocalThriftgo, "local-thriftgo", false, + "Use local thriftgo exec instead of kitex embedded thriftgo. This is mainly used for debugging, you need to ensure that thriftgo is installed correctly.") f.Var(&a.BuiltinTpl, "tpl", "Specify kitex built-in template.") f.BoolVar(&a.StreamX, "streamx", false, "Generate streaming code with streamx interface", @@ -335,7 +329,7 @@ func (a *Arguments) checkPath(curpath string) error { } a.PackagePrefix = path.Join(a.ModuleName, filepath.ToSlash(refPath), genPath) } else { - if err := initGoMod(a.ModuleName); err != nil { + if err := initGoMod(curpath, a.ModuleName); err != nil { return fmt.Errorf("init go mod failed: %w", err) } a.PackagePrefix = path.Join(a.ModuleName, genPath) @@ -458,93 +452,20 @@ func (a *Arguments) BuildCmd(out io.Writer) (*exec.Cmd, error) { return cmd, nil } -// ValidateCMD check if the path exists and if the version is satisfied +// ValidateCMD check if the path exists and if the version is satisfied. Thriftgo is embedded, only validate protoc. func ValidateCMD(path, idlType string) error { - // check if the path exists - if _, err := os.Stat(path); err != nil { - tool := "thriftgo" - if idlType == "protobuf" { - tool = "protoc" - } - - if idlType == "thrift" { - _, err = runCommand("go install github.com/cloudwego/thriftgo@latest") - if err != nil { - return fmt.Errorf("[ERROR] %s is also unavailable, please install %s first.\n"+ - "Refer to https://github.com/cloudwego/thriftgo, or simple run:\n"+ - " go install -v github.com/cloudwego/thriftgo@latest", path, tool) - } - } else { - return fmt.Errorf("[ERROR] %s is also unavailable, please install %s first.\n"+ - "Refer to https://github.com/protocolbuffers/protobuf", path, tool) + if idlType == "protobuf" { + if _, err := os.Stat(path); err != nil { + return fmt.Errorf("[ERROR] %s is also unavailable, please install protoc first.\n"+ + "Refer to https://github.com/protocolbuffers/protobuf", path) } } - - // check if the version is satisfied if idlType == "thrift" { - // run `thriftgo -versions and get the output - cmd := exec.Command(path, "-version") - out, err := cmd.CombinedOutput() - if err != nil { - return fmt.Errorf("failed to get thriftgo version: %s", err.Error()) - } - if !strings.HasPrefix(string(out), "thriftgo ") { - return fmt.Errorf("thriftgo -version returns '%s', please reinstall thriftgo first", string(out)) - } - version := strings.Replace(strings.TrimSuffix(string(out), "\n"), "thriftgo ", "", 1) - if !versionSatisfied(version, requiredThriftGoVersion) { - _, err = runCommand("go install github.com/cloudwego/thriftgo@latest") - if err != nil { - return fmt.Errorf("[ERROR] thriftgo version(%s) not satisfied, please install version >= %s", - version, requiredThriftGoVersion) - } - } - return nil + log.Warnf("You are using local thriftgo: %s. Please make sure the version is matched with kitex tool by yourself.", path) } return nil } -var versionSuffixPattern = regexp.MustCompile(`-.*$`) - -func removeVersionPrefixAndSuffix(version string) string { - version = strings.TrimPrefix(version, "v") - version = strings.TrimSuffix(version, "\n") - version = versionSuffixPattern.ReplaceAllString(version, "") - return version -} - -func versionSatisfied(current, required string) bool { - currentSegments := strings.SplitN(removeVersionPrefixAndSuffix(current), ".", 3) - requiredSegments := strings.SplitN(removeVersionPrefixAndSuffix(required), ".", 3) - - requiredHasSuffix := versionSuffixPattern.MatchString(required) - if requiredHasSuffix { - return false // required version should be a formal version - } - - for i := 0; i < 3; i++ { - var currentSeg, minimalSeg int - var err error - if currentSeg, err = strconv.Atoi(currentSegments[i]); err != nil { - log.Warnf("invalid current version: %s, seg=%v, err=%v", current, currentSegments[i], err) - return false - } - if minimalSeg, err = strconv.Atoi(requiredSegments[i]); err != nil { - log.Warnf("invalid required version: %s, seg=%v, err=%v", required, requiredSegments[i], err) - return false - } - if currentSeg > minimalSeg { - return true - } else if currentSeg < minimalSeg { - return false - } - } - if currentHasSuffix := versionSuffixPattern.MatchString(current); currentHasSuffix { - return false - } - return true -} - // LookupTool returns the compiler path found in $PATH; if not found, returns $GOPATH/bin/$tool func LookupTool(idlType, compilerPath string) string { tool := "thriftgo" @@ -568,7 +489,7 @@ func LookupTool(idlType, compilerPath string) string { return path } -func initGoMod(module string) error { +func initGoMod(curpath, module string) error { if util.Exists("go.mod") { return nil } @@ -577,6 +498,7 @@ func initGoMod(module string) error { return err } cmd := &exec.Cmd{ + Dir: curpath, Path: pathToGo, Args: []string{"go", "mod", "init", module}, Stdin: os.Stdin, @@ -585,21 +507,3 @@ func initGoMod(module string) error { } return cmd.Run() } - -func runCommand(input string) (string, error) { - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - arr := strings.Split(input, " ") - var cmd *exec.Cmd - if len(arr) > 1 { - cmd = exec.CommandContext(ctx, arr[0], arr[1:]...) - } else { - cmd = exec.CommandContext(ctx, arr[0]) - } - output, err := cmd.CombinedOutput() - if err != nil { - return "", err - } - result := strings.TrimSpace(string(output)) - return result, nil -} diff --git a/tool/cmd/kitex/args/args_test.go b/tool/cmd/kitex/args/args_test.go index 93d0675ea4..77774fb551 100644 --- a/tool/cmd/kitex/args/args_test.go +++ b/tool/cmd/kitex/args/args_test.go @@ -24,37 +24,6 @@ import ( "github.com/cloudwego/kitex/internal/test" ) -func Test_versionSatisfied(t *testing.T) { - t.Run("lower", func(t *testing.T) { - test.Assert(t, !versionSatisfied("v1.0.0", "v1.0.1")) - test.Assert(t, !versionSatisfied("v1.0.1", "v1.1.0")) - test.Assert(t, !versionSatisfied("v1.1.1", "v2.0.0")) - }) - - t.Run("equal", func(t *testing.T) { - test.Assert(t, versionSatisfied("v1.2.3", "v1.2.3")) - }) - - t.Run("higher", func(t *testing.T) { - test.Assert(t, versionSatisfied("v1.2.4", "v1.2.3")) - test.Assert(t, versionSatisfied("v1.2.3", "v1.1.4")) - test.Assert(t, versionSatisfied("v2.2.3", "v1.3.4")) - }) - - t.Run("suffix", func(t *testing.T) { - test.Assert(t, !versionSatisfied("v1.2.3", "v1.2.3-rc1")) // required shouldn't have suffix - test.Assert(t, !versionSatisfied("v1.2.3-rc1", "v1.2.3")) // suffix means lower - }) - t.Run("no prefix", func(t *testing.T) { - test.Assert(t, versionSatisfied("v1.2.3", "1.2.3")) - test.Assert(t, versionSatisfied("1.2.3", "v1.2.3")) - test.Assert(t, versionSatisfied("1.2.3", "1.2.3")) - test.Assert(t, !versionSatisfied("v1.2.3", "1.2.4")) - test.Assert(t, !versionSatisfied("1.2.3", "v1.2.4")) - test.Assert(t, !versionSatisfied("1.2.3", "1.2.4")) - }) -} - func Test_refGoSrcPath(t *testing.T) { gopath, _ := os.Getwd() t.Setenv("GOPATH", gopath) diff --git a/tool/cmd/kitex/args/version_requirements.go b/tool/cmd/kitex/args/version_requirements.go deleted file mode 100644 index 72ef19ce26..0000000000 --- a/tool/cmd/kitex/args/version_requirements.go +++ /dev/null @@ -1,17 +0,0 @@ -// Copyright 2023 CloudWeGo Authors -// -// 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 args - -var requiredThriftGoVersion = "v0.3.15" diff --git a/tool/cmd/kitex/main.go b/tool/cmd/kitex/main.go index de4dcf0c44..38e8532c55 100644 --- a/tool/cmd/kitex/main.go +++ b/tool/cmd/kitex/main.go @@ -105,9 +105,6 @@ func main() { if args.IDLType == "thrift" && !args.LocalThriftgo { err = sdk.InvokeThriftgoBySDK(curpath, cmd) } else { - if args.IDLType == "thrift" { - log.Warn("You are using local thriftgo. Please make sure the version is matched with kitex tool.") - } err = kargs.ValidateCMD(cmd.Path, args.IDLType) if err != nil { log.Warn(err) From 293c68454185ce6bc4b8ae7622ee519c89900cb0 Mon Sep 17 00:00:00 2001 From: Marina Sakai <118230951+Marina-Sakai@users.noreply.github.com> Date: Wed, 12 Feb 2025 14:38:38 +0800 Subject: [PATCH 23/35] chore(generic): wait until server starts in tests (#1694) --- pkg/generic/grpcjson_test/generic_test.go | 4 +++- pkg/generic/grpcjsonpb_test/generic_test.go | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/pkg/generic/grpcjson_test/generic_test.go b/pkg/generic/grpcjson_test/generic_test.go index b2ce5053a6..68ce6732b1 100644 --- a/pkg/generic/grpcjson_test/generic_test.go +++ b/pkg/generic/grpcjson_test/generic_test.go @@ -174,7 +174,9 @@ func getJsonThriftGeneric(idl string, enableDynamicgo bool) (generic.Generic, er func initMockTestServer(handler kt.TestService, address string) server.Server { addr, _ := net.ResolveTCPAddr("tcp", address) - return newMockServer(handler, addr) + svr := newMockServer(handler, addr) + test.WaitServerStart(addr.String()) + return svr } func testClientStreaming(t *testing.T, ctx context.Context, cli genericclient.Client) { diff --git a/pkg/generic/grpcjsonpb_test/generic_test.go b/pkg/generic/grpcjsonpb_test/generic_test.go index 2f10b545af..bed8c479d8 100644 --- a/pkg/generic/grpcjsonpb_test/generic_test.go +++ b/pkg/generic/grpcjsonpb_test/generic_test.go @@ -158,7 +158,9 @@ func initStreamingClient(t *testing.T, ctx context.Context, addr, idl string, cl func initMockTestServer(handler mock.Mock, address string) server.Server { addr, _ := net.ResolveTCPAddr("tcp", address) - return newMockTestServer(handler, addr) + svr := newMockTestServer(handler, addr) + test.WaitServerStart(addr.String()) + return svr } func Test_invocationContainsPackage(t *testing.T) { From 6dbce3da0f9ab407e35781d4c8db4d8ef23b362e Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Wed, 12 Feb 2025 14:40:21 +0800 Subject: [PATCH 24/35] fix(server): log recover result (#1693) --- pkg/remote/trans/default_server_handler.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/remote/trans/default_server_handler.go b/pkg/remote/trans/default_server_handler.go index 7a7c681804..33791d9cc8 100644 --- a/pkg/remote/trans/default_server_handler.go +++ b/pkg/remote/trans/default_server_handler.go @@ -158,11 +158,11 @@ func (t *svrTransHandler) OnRead(ctx context.Context, conn net.Conn) (err error) if conn != nil { ri := rpcinfo.GetRPCInfo(ctx) rService, rAddr := getRemoteInfo(ri, conn) - klog.CtxErrorf(ctx, "KITEX: panic happened, remoteAddress=%s, remoteService=%s, error=%v\nstack=%s", rAddr, rService, panicErr, stack) + klog.CtxErrorf(ctx, "KITEX: panic happened, remoteAddress=%s, remoteService=%s, error=%v\nstack=%s", rAddr, rService, r, stack) } else { - klog.CtxErrorf(ctx, "KITEX: panic happened, error=%v\nstack=%s", panicErr, stack) + klog.CtxErrorf(ctx, "KITEX: panic happened, error=%v\nstack=%s", r, stack) } - panicErr = kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in OnRead] %s", panicErr), stack) + panicErr = kerrors.ErrPanic.WithCauseAndStack(fmt.Errorf("[happened in OnRead] %v", r), stack) if err == nil { err = panicErr } From 885b05aaae6b5a86b745194e2fee02c58e32f873 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Thu, 13 Feb 2025 14:39:58 +0800 Subject: [PATCH 25/35] feat(gRPC): add monitor closeStreamTask for cancelled gRPC client stream (#1650) --- .github/workflows/tests.yml | 8 +- pkg/remote/trans/nphttp2/grpc/http2_client.go | 42 +++++++++++ .../trans/nphttp2/grpc/transport_test.go | 53 +++++++++++++- pkg/utils/sharedticker.go | 47 +++++++++--- pkg/utils/sharedticker_test.go | 73 ++++++++++--------- 5 files changed, 173 insertions(+), 50 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index b9de72b07d..6c43045287 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -10,7 +10,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v5 with: - go-version: stable + go-version: "1.23" cache: false - name: Scenario Tests run: | @@ -28,7 +28,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v5 with: - go-version: stable + go-version: "1.23" - name: Benchmark # we only use this CI to verify bench code works # setting benchtime=100ms is saving our time... @@ -71,7 +71,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v5 with: - go-version: stable + go-version: "1.23" - name: Prepare run: | go install github.com/cloudwego/thriftgo@main @@ -100,7 +100,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v5 with: - go-version: stable + go-version: "1.23" cache: false # don't use cache for self-hosted runners - name: Windows compatibility test run: go test -run=^$ ./... diff --git a/pkg/remote/trans/nphttp2/grpc/http2_client.go b/pkg/remote/trans/nphttp2/grpc/http2_client.go index f2d6a4ccfd..37ac2f32bc 100644 --- a/pkg/remote/trans/nphttp2/grpc/http2_client.go +++ b/pkg/remote/trans/nphttp2/grpc/http2_client.go @@ -46,8 +46,15 @@ import ( "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/metadata" "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/peer" "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/status" + "github.com/cloudwego/kitex/pkg/utils" ) +// ticker is used to manage closeStreamTask. +// it triggers and cleans up actively cancelled streams every 5s. +// Streaming QPS is generally not too high, if there is a requirement for timeliness, then consider making it configurable. +// To reduce the overhead of goroutines in a multi-connection scenario, use the Sync SharedTicker +var ticker = utils.NewSyncSharedTicker(5 * time.Second) + // http2Client implements the ClientTransport interface with HTTP2. type http2Client struct { lastRead int64 // Keep this field 64-bit aligned. Accessed atomically. @@ -200,6 +207,13 @@ func newHTTP2Client(ctx context.Context, conn net.Conn, opts ConnectOptions, } } t.loopy = newLoopyWriter(clientSide, t.framer, t.controlBuf, t.bdpEst) + task := &closeStreamTask{t: t} + t.onClose = func() { + onClose() + // when http2Client has been closed, remove this task + ticker.Delete(task) + } + ticker.Add(task) // Start the reader goroutine for incoming message. Each transport has // a dedicated goroutine which reads HTTP2 frame from network. Then it @@ -272,6 +286,34 @@ func newHTTP2Client(ctx context.Context, conn net.Conn, opts ConnectOptions, return t, nil } +// closeStreamTask is used to clean up streams that have been actively cancelled by users +type closeStreamTask struct { + t *http2Client + toCloseStreams []*Stream +} + +func (task closeStreamTask) Tick() { + trans := task.t + trans.mu.Lock() + for _, stream := range trans.activeStreams { + select { + // judge whether stream has been canceled + case <-stream.Context().Done(): + task.toCloseStreams = append(task.toCloseStreams, stream) + default: + } + } + trans.mu.Unlock() + + for i, stream := range task.toCloseStreams { + // uniformly converted to status error + sErr := ContextErr(stream.Context().Err()) + trans.closeStream(stream, sErr, true, http2.ErrCodeCancel, status.Convert(sErr), nil, false) + task.toCloseStreams[i] = nil + } + task.toCloseStreams = task.toCloseStreams[:0] +} + type preAllocatedStreamFields struct { recvBuffer *recvBuffer writeQuota *writeQuota diff --git a/pkg/remote/trans/nphttp2/grpc/transport_test.go b/pkg/remote/trans/nphttp2/grpc/transport_test.go index 9f584386bb..96b84a38e8 100644 --- a/pkg/remote/trans/nphttp2/grpc/transport_test.go +++ b/pkg/remote/trans/nphttp2/grpc/transport_test.go @@ -46,6 +46,7 @@ import ( "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/grpc/grpcframe" "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/grpc/testutils" "github.com/cloudwego/kitex/pkg/remote/trans/nphttp2/status" + "github.com/cloudwego/kitex/pkg/utils" ) type server struct { @@ -99,6 +100,7 @@ const ( pingpong gracefulShutdown + cancel ) func (h *testStreamHandler) handleStreamAndNotify(s *Stream) { @@ -317,6 +319,17 @@ func (h *testStreamHandler) gracefulShutdown(t *testing.T, s *Stream) { test.Assert(t, st.Code() == codes.Unavailable, st) } +func (h *testStreamHandler) handleStreamCancel(t *testing.T, s *Stream) { + header := make([]byte, 5) + _, err := s.Read(header) + test.Assert(t, err != nil, err) + st, ok := status.FromError(err) + test.Assert(t, ok) + test.Assert(t, st.Code() == codes.Canceled, st.Code()) + test.Assert(t, strings.Contains(st.Message(), "transport: RSTStream Frame received with error code"), st.Message()) + close(h.srv.srvReady) +} + // start starts server. Other goroutines should block on s.readyChan for further operations. func (s *server) start(t *testing.T, port int, serverConfig *ServerConfig, ht hType) { // 创建 listener @@ -416,6 +429,12 @@ func (s *server) start(t *testing.T, port int, serverConfig *ServerConfig, ht hT }() }, func(ctx context.Context, method string) context.Context { return ctx }) }() + case cancel: + go transport.HandleStreams(func(s *Stream) { + go h.handleStreamCancel(t, s) + }, func(ctx context.Context, method string) context.Context { + return ctx + }) default: go func() { transport.HandleStreams(func(s *Stream) { @@ -1527,8 +1546,11 @@ func TestEncodingRequiredStatus(t *testing.T) { return } opts := Options{Last: true} - if err := ct.Write(s, nil, expectedRequest, &opts); err != nil && err != errStreamDone { - t.Fatalf("Failed to write the request: %v", err) + if err := ct.Write(s, nil, expectedRequest, &opts); err != nil { + // in case server-side returns earlier, Write also returns a status error. + if err != errStatusStreamDone || !testutils.StatusErrEqual(s.Status().Err(), encodingTestStatus.Err()) { + t.Fatalf("Failed to write the request: %v", err) + } } p := make([]byte, http2MaxFrameLen) if _, err := s.trReader.(*transportReader).Read(p); err != io.EOF { @@ -2137,3 +2159,30 @@ func Test_isIgnorable(t *testing.T) { test.Assert(t, isIgnorable(ErrConnClosing)) } + +func Test_closeStreamTask(t *testing.T) { + // replace ticker to reduce test time + ticker = utils.NewSyncSharedTicker(10 * time.Millisecond) + server, ct := setUp(t, 0, math.MaxUint32, cancel) + callHdr := &CallHdr{ + Host: "localhost", + Method: "foo.Small", + } + ctx, cancel := context.WithCancel(context.Background()) + stream, err := ct.NewStream(ctx, callHdr) + if err != nil { + t.Fatalf("failed to open stream: %v", err) + } + cancel() + // wait for server receiving RstStream Frame + <-server.srvReady + state := stream.getState() + test.Assert(t, state == streamDone, state) + ct.mu.Lock() + streamNums := len(ct.activeStreams) + ct.mu.Unlock() + test.Assert(t, streamNums == 0, streamNums) + + ct.Close(errSelfCloseForTest) + server.stop() +} diff --git a/pkg/utils/sharedticker.go b/pkg/utils/sharedticker.go index ee0f69b881..2eeb15c396 100644 --- a/pkg/utils/sharedticker.go +++ b/pkg/utils/sharedticker.go @@ -37,12 +37,21 @@ func NewSharedTicker(interval time.Duration) *SharedTicker { } } +// NewSyncSharedTicker constructs a SharedTicker with specified interval. +// The Tick method will run tasks synchronously. +func NewSyncSharedTicker(interval time.Duration) *SharedTicker { + t := NewSharedTicker(interval) + t.sync = true + return t +} + type SharedTicker struct { sync.Mutex started bool Interval time.Duration tasks map[TickerTask]struct{} stopChan chan struct{} + sync bool // identify whether Tick runs task synchronously } func (t *SharedTicker) Add(b TickerTask) { @@ -82,25 +91,41 @@ func (t *SharedTicker) Closed() bool { } func (t *SharedTicker) Tick(interval time.Duration) { - var wg sync.WaitGroup ticker := time.NewTicker(interval) defer ticker.Stop() for { select { case <-ticker.C: - t.Lock() - for b := range t.tasks { - wg.Add(1) - task := b - gofunc.GoFunc(context.Background(), func() { - defer wg.Done() - task.Tick() - }) + if t.sync { + t.syncExec() + } else { + t.asyncExec() } - t.Unlock() - wg.Wait() case <-t.stopChan: return } } } + +func (t *SharedTicker) syncExec() { + t.Lock() + for task := range t.tasks { + task.Tick() + } + t.Unlock() +} + +func (t *SharedTicker) asyncExec() { + var wg sync.WaitGroup + t.Lock() + for b := range t.tasks { + wg.Add(1) + task := b + gofunc.GoFunc(context.Background(), func() { + defer wg.Done() + task.Tick() + }) + } + t.Unlock() + wg.Wait() +} diff --git a/pkg/utils/sharedticker_test.go b/pkg/utils/sharedticker_test.go index 8dd636bd2e..ded236de28 100644 --- a/pkg/utils/sharedticker_test.go +++ b/pkg/utils/sharedticker_test.go @@ -32,35 +32,40 @@ func TestSharedTickerAdd(t *testing.T) { rt := mockutils.NewMockTickerTask(ctrl) rt.EXPECT().Tick().AnyTimes() - st := NewSharedTicker(1) - st.Add(nil) - test.Assert(t, len(st.tasks) == 0) - st.Add(rt) - test.Assert(t, len(st.tasks) == 1) - test.Assert(t, !st.Closed()) + + sts := []*SharedTicker{NewSharedTicker(1), NewSyncSharedTicker(1)} + for _, st := range sts { + st.Add(nil) + test.Assert(t, len(st.tasks) == 0) + st.Add(rt) + test.Assert(t, len(st.tasks) == 1) + test.Assert(t, !st.Closed()) + } } func TestSharedTickerDeleteAndClose(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() - st := NewSharedTicker(1) - var ( - num = 10 - tasks = make([]TickerTask, num) - ) - for i := 0; i < num; i++ { - rt := mockutils.NewMockTickerTask(ctrl) - rt.EXPECT().Tick().AnyTimes() - tasks[i] = rt - st.Add(rt) - } - test.Assert(t, len(st.tasks) == num) - for i := 0; i < num; i++ { - st.Delete(tasks[i]) + sts := []*SharedTicker{NewSharedTicker(1), NewSyncSharedTicker(1)} + for _, st := range sts { + var ( + num = 10 + tasks = make([]TickerTask, num) + ) + for i := 0; i < num; i++ { + rt := mockutils.NewMockTickerTask(ctrl) + rt.EXPECT().Tick().AnyTimes() + tasks[i] = rt + st.Add(rt) + } + test.Assert(t, len(st.tasks) == num) + for i := 0; i < num; i++ { + st.Delete(tasks[i]) + } + test.Assert(t, len(st.tasks) == 0) + test.Assert(t, st.Closed()) } - test.Assert(t, len(st.tasks) == 0) - test.Assert(t, st.Closed()) } func TestSharedTickerTick(t *testing.T) { @@ -68,16 +73,18 @@ func TestSharedTickerTick(t *testing.T) { defer ctrl.Finish() duration := 100 * time.Millisecond - st := NewSharedTicker(duration) - var ( - num = 10 - tasks = make([]TickerTask, num) - ) - for i := 0; i < num; i++ { - rt := mockutils.NewMockTickerTask(ctrl) - rt.EXPECT().Tick().MinTimes(1) // all tasks should be refreshed once during the test - tasks[i] = rt - st.Add(rt) + sts := []*SharedTicker{NewSharedTicker(duration), NewSyncSharedTicker(duration)} + for _, st := range sts { + var ( + num = 10 + tasks = make([]TickerTask, num) + ) + for i := 0; i < num; i++ { + rt := mockutils.NewMockTickerTask(ctrl) + rt.EXPECT().Tick().MinTimes(1) // all tasks should be refreshed once during the test + tasks[i] = rt + st.Add(rt) + } + time.Sleep(150 * time.Millisecond) } - time.Sleep(150 * time.Millisecond) } From ebac2b07e8e406ec5a5e6c634d537cf36f40a777 Mon Sep 17 00:00:00 2001 From: Li2CO3 Date: Thu, 13 Feb 2025 16:08:01 +0800 Subject: [PATCH 26/35] optimize(tool): optimize fastcodec->frugal replacement (#1696) --- tool/cmd/kitex/args/args.go | 3 +- tool/internal_pkg/generator/generator.go | 1 - tool/internal_pkg/generator/generator_test.go | 2 +- .../pluginmode/thriftgo/patcher.go | 29 +++++++++---------- .../pluginmode/thriftgo/plugin.go | 1 - .../pluginmode/thriftgo/struct_tpl.go | 11 +++++++ 6 files changed, 27 insertions(+), 20 deletions(-) diff --git a/tool/cmd/kitex/args/args.go b/tool/cmd/kitex/args/args.go index c4ad137549..37548abc30 100644 --- a/tool/cmd/kitex/args/args.go +++ b/tool/cmd/kitex/args/args.go @@ -131,8 +131,7 @@ func (a *Arguments) buildFlags(version string) *flag.FlagSet { "Generate streaming code with streamx interface", ) - f.BoolVar(&a.GenFrugal, "gen-frugal", false, `Gen frugal codec for those structs with (go.codec="frugal")`) - f.Var(&a.FrugalStruct, "frugal-struct", "Gen frugal codec for given struct") + f.Var(&a.FrugalStruct, "frugal-struct", "Replace fastCodec code to frugal. Use `-frugal-struct @all` for all, `-frugal-struct @auto` for annotated structs (go.codec=\"frugal\"), or specify multiple structs (e.g., `-frugal-struct A -frugal-struct B`).") f.BoolVar(&a.NoRecurse, "no-recurse", false, `Don't generate thrift files recursively, just generate the given file.'`) diff --git a/tool/internal_pkg/generator/generator.go b/tool/internal_pkg/generator/generator.go index 4021c29591..ae686f20e1 100644 --- a/tool/internal_pkg/generator/generator.go +++ b/tool/internal_pkg/generator/generator.go @@ -148,7 +148,6 @@ type Config struct { Rapid bool LocalThriftgo bool - GenFrugal bool FrugalStruct util.StringSlice NoRecurse bool diff --git a/tool/internal_pkg/generator/generator_test.go b/tool/internal_pkg/generator/generator_test.go index e7806c2fa0..4fc1b12658 100644 --- a/tool/internal_pkg/generator/generator_test.go +++ b/tool/internal_pkg/generator/generator_test.go @@ -67,7 +67,7 @@ func TestConfig_Pack(t *testing.T) { { name: "some", fields: fields{Features: []feature{feature(999)}, ThriftPluginTimeLimit: 30 * time.Second}, - wantRes: []string{"Verbose=false", "GenerateMain=false", "GenerateInvoker=false", "Version=", "NoFastAPI=false", "ModuleName=", "ServiceName=", "Use=", "IDLType=", "Includes=", "ThriftOptions=", "ProtobufOptions=", "Hessian2Options=", "IDL=", "OutputPath=", "PackagePrefix=", "CombineService=false", "CopyIDL=false", "ProtobufPlugins=", "Features=999", "FrugalPretouch=false", "ThriftPluginTimeLimit=30s", "CompilerPath=", "ExtensionFile=", "Record=false", "RecordCmd=", "TemplateDir=", "GenPath=", "DeepCopyAPI=false", "Protocol=", "HandlerReturnKeepResp=false", "NoDependencyCheck=false", "Rapid=false", "LocalThriftgo=false", "GenFrugal=false", "FrugalStruct=", "NoRecurse=false", "BuiltinTpl=", "StreamX=false"}, + wantRes: []string{"Verbose=false", "GenerateMain=false", "GenerateInvoker=false", "Version=", "NoFastAPI=false", "ModuleName=", "ServiceName=", "Use=", "IDLType=", "Includes=", "ThriftOptions=", "ProtobufOptions=", "Hessian2Options=", "IDL=", "OutputPath=", "PackagePrefix=", "CombineService=false", "CopyIDL=false", "ProtobufPlugins=", "Features=999", "FrugalPretouch=false", "ThriftPluginTimeLimit=30s", "CompilerPath=", "ExtensionFile=", "Record=false", "RecordCmd=", "TemplateDir=", "GenPath=", "DeepCopyAPI=false", "Protocol=", "HandlerReturnKeepResp=false", "NoDependencyCheck=false", "Rapid=false", "LocalThriftgo=false", "FrugalStruct=", "NoRecurse=false", "BuiltinTpl=", "StreamX=false"}, }, } for _, tt := range tests { diff --git a/tool/internal_pkg/pluginmode/thriftgo/patcher.go b/tool/internal_pkg/pluginmode/thriftgo/patcher.go index 4dd5f8f6d0..11174f0085 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/patcher.go +++ b/tool/internal_pkg/pluginmode/thriftgo/patcher.go @@ -61,32 +61,31 @@ type patcher struct { protocol string handlerReturnKeepResp bool - genFrugal bool frugalStruct []string fileTpl *template.Template libs map[string]string } +// UseFrugalForStruct judge if we need to replace fastCodec to frugal implementation. It's related to the argument '-frugal-struct'. func (p *patcher) UseFrugalForStruct(st *golang.StructLike) bool { - if len(p.frugalStruct) > 0 { - for _, structName := range p.frugalStruct { - if st.GetName() == structName { - return true - } - } - return false - } else { - if !p.genFrugal { - return false + // '@all' matches all structs + // '@auto' matches those with annotation (go.codec="frugal") + // otherwise, check if the given name matches the struct's name + for _, structName := range p.frugalStruct { + if structName == "@all" || st.GetName() == structName { + return true } - for _, anno := range st.Annotations { - if anno.GetKey() == "go.codec" && len(anno.GetValues()) > 0 && anno.GetValues()[0] == "frugal" { - return true + if structName == "@auto" { + // find annotation go.codec="frugal" + for _, anno := range st.Annotations { + if anno.GetKey() == "go.codec" && len(anno.GetValues()) > 0 && anno.GetValues()[0] == "frugal" { + return true + } } } - return false } + return false } func (p *patcher) UseLib(path, alias string) string { diff --git a/tool/internal_pkg/pluginmode/thriftgo/plugin.go b/tool/internal_pkg/pluginmode/thriftgo/plugin.go index 2084207aa8..75e877aa29 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/plugin.go +++ b/tool/internal_pkg/pluginmode/thriftgo/plugin.go @@ -147,7 +147,6 @@ func HandleRequest(req *plugin.Request) *plugin.Response { deepCopyAPI: conv.Config.DeepCopyAPI, protocol: conv.Config.Protocol, handlerReturnKeepResp: conv.Config.HandlerReturnKeepResp, - genFrugal: conv.Config.GenFrugal, frugalStruct: conv.Config.FrugalStruct, } // for cmd without setting -module diff --git a/tool/internal_pkg/pluginmode/thriftgo/struct_tpl.go b/tool/internal_pkg/pluginmode/thriftgo/struct_tpl.go index 395d84f990..5b5583834c 100644 --- a/tool/internal_pkg/pluginmode/thriftgo/struct_tpl.go +++ b/tool/internal_pkg/pluginmode/thriftgo/struct_tpl.go @@ -150,6 +150,7 @@ RequiredFieldNotSetError: const StructLikeFastReadField = ` {{define "StructLikeFastReadField"}} +{{- if not (UseFrugalForStruct .) }} {{- $TypeName := .GoName}} {{- range .Fields}} {{$FieldName := .GoName}} @@ -177,6 +178,7 @@ func (p *{{$TypeName}}) FastReadField{{Str .ID}}(buf []byte) (int, error) { return offset, nil } {{- end}}{{/* range .Fields */}} +{{- end}}{{/* if not (UseFrugalForStruct .) */}} {{- end}}{{/* define "StructLikeFastReadField" */}} ` @@ -265,6 +267,10 @@ const StructLikeLength = ` {{define "StructLikeLength"}} {{- $TypeName := .GoName}} func (p *{{$TypeName}}) BLength() int { +{{- if UseFrugalForStruct .}} + {{- UseLib "github.com/cloudwego/frugal" "frugal"}} + return frugal.EncodedSize(p) +{{- else}} l := 0 {{- if eq .Category "union"}} var c int @@ -289,12 +295,14 @@ func (p *{{$TypeName}}) BLength() int { CountSetFieldsError: panic(fmt.Errorf("%T write union: exactly one field must be set (%d set).", p, c)) {{- end}} +{{- end}}{{/* frugal */}} } {{- end}}{{/* define "StructLikeLength" */}} ` const StructLikeFastWriteField = ` {{define "StructLikeFastWriteField"}} +{{- if not (UseFrugalForStruct .) }} {{- $TypeName := .GoName}} {{- range .Fields}} {{- $FieldName := .GoName}} @@ -333,11 +341,13 @@ func (p *{{$TypeName}}) fastWriteField{{Str .ID}}(buf []byte, w thrift.NocopyWri return offset } {{end}}{{/* range .Fields */}} +{{- end}}{{/* if not (UseFrugalForStruct .) */}} {{- end}}{{/* define "StructLikeFastWriteField" */}} ` const StructLikeFieldLength = ` {{define "StructLikeFieldLength"}} +{{- if not (UseFrugalForStruct .) }} {{- $TypeName := .GoName}} {{- range .Fields}} {{- $FieldName := .GoName}} @@ -376,6 +386,7 @@ func (p *{{$TypeName}}) field{{Str .ID}}Length() int { return l } {{end}}{{/* range .Fields */}} +{{- end}}{{/* if not (UseFrugalForStruct .) */}} {{- end}}{{/* define "StructLikeFieldLength" */}} ` From 5f0296e4e86f282b53e5bea18efb09dde02f4860 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Fri, 14 Feb 2025 11:02:01 +0800 Subject: [PATCH 27/35] fix(ttstream): pingpong method refers to server interface defined in Kitex generation code when streamx is enabled and there are other streaming methods (#1698) --- tool/internal_pkg/tpl/service.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/tool/internal_pkg/tpl/service.go b/tool/internal_pkg/tpl/service.go index e4bf26f069..4da2f47ec8 100644 --- a/tool/internal_pkg/tpl/service.go +++ b/tool/internal_pkg/tpl/service.go @@ -144,6 +144,7 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami return svcInfo } +{{- $referToKitexServerInterface := and .StreamX .HasStreaming}} {{range .AllMethods}} {{- $isStreaming := or .ClientStreaming .ServerStreaming}} {{- $streamingUnary := (eq .StreamingMode "unary")}} @@ -153,6 +154,10 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami {{- $bidiSide := and .ClientStreaming .ServerStreaming}} {{- $arg := ""}} {{- $handlerFunc := ""}} +{{- $serverInterfaceName := printf "%s.%s" .PkgRefName .ServiceName }} + {{- if $referToKitexServerInterface}} + {{- $serverInterfaceName = .ServiceName}}{{/* when streamx is enabled and there are streaming methods, refer to Server Interface defined in service/server.go */}} + {{- end}} {{- $mode := ""}} {{- if $streamingUnary -}} {{- $mode = "serviceinfo.StreamingUnary" }} {{- $handlerFunc = "InvokeUnaryHandler" }} {{- else if $serverSide -}} {{- $mode = "serviceinfo.StreamingServer" }} {{- $handlerFunc = "InvokeServerStreamHandler" }} @@ -228,8 +233,8 @@ func {{LowerFirst .Name}}Handler(ctx context.Context, handler interface{}, arg, {{- end}} {{if gt .ArgsLength 0}}realArg := {{else}}_ = {{end}}arg.(*{{if not .GenArgResultStruct}}{{.PkgRefName}}.{{end}}{{.ArgStructName}}) {{if or (not .Void) .Exceptions}}realResult := result.(*{{if not .GenArgResultStruct}}{{.PkgRefName}}.{{end}}{{.ResStructName}}){{end}} - {{if .Void}}err := handler.({{.PkgRefName}}.{{.ServiceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) - {{else}}success, err := handler.({{.PkgRefName}}.{{.ServiceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) + {{if .Void}}err := handler.({{$serverInterfaceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) + {{else}}success, err := handler.({{$serverInterfaceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) {{end -}} if err != nil { {{- if $HandlerReturnKeepResp }} From 4e122b287721c89f0688e31c64dfedb72c407b96 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Fri, 14 Feb 2025 13:57:51 +0800 Subject: [PATCH 28/35] ci: uses codecov (#1699) --- .github/workflows/tests.yml | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index 6c43045287..f968144692 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -34,7 +34,7 @@ jobs: # setting benchtime=100ms is saving our time... run: go test -bench=. -benchmem -run=none ./... -benchtime=100ms - compatibility-test-x64: + unit-test-x64: strategy: matrix: go: [ "1.18", "1.19", "1.20", "1.21", "1.22", "1.23" ] @@ -49,7 +49,7 @@ jobs: - name: Unit Test run: go test -race ./... - compatibility-test-arm: + unit-test-arm: strategy: matrix: go: [ "1.18", "1.19", "1.20", "1.21", "1.22", "1.23" ] @@ -104,3 +104,18 @@ jobs: cache: false # don't use cache for self-hosted runners - name: Windows compatibility test run: go test -run=^$ ./... + + codecov: + needs: [ unit-scenario-test, unit-test-x64, unit-test-arm ] + runs-on: [ Linux, X64 ] + steps: + - uses: actions/checkout@v4 + - name: Set up Go + uses: actions/setup-go@v5 + with: + go-version: "1.23" + cache: false # don't use cache for self-hosted runners + - name: Run coverage + run: go test -race -coverprofile=coverage.out -covermode=atomic ./... + - name: Upload coverage to Codecov + uses: codecov/codecov-action@v5 From 5941cd74106059a14a66b0acee9e728ecb372984 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Fri, 14 Feb 2025 14:55:28 +0800 Subject: [PATCH 29/35] Revert "fix(ttstream): pingpong method refers to server interface defined in Kitex generation code when streamx is enabled and there are other streaming methods" (#1700) --- tool/internal_pkg/tpl/service.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tool/internal_pkg/tpl/service.go b/tool/internal_pkg/tpl/service.go index 4da2f47ec8..e4bf26f069 100644 --- a/tool/internal_pkg/tpl/service.go +++ b/tool/internal_pkg/tpl/service.go @@ -144,7 +144,6 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami return svcInfo } -{{- $referToKitexServerInterface := and .StreamX .HasStreaming}} {{range .AllMethods}} {{- $isStreaming := or .ClientStreaming .ServerStreaming}} {{- $streamingUnary := (eq .StreamingMode "unary")}} @@ -154,10 +153,6 @@ func newServiceInfo(hasStreaming bool, keepStreamingMethods bool, keepNonStreami {{- $bidiSide := and .ClientStreaming .ServerStreaming}} {{- $arg := ""}} {{- $handlerFunc := ""}} -{{- $serverInterfaceName := printf "%s.%s" .PkgRefName .ServiceName }} - {{- if $referToKitexServerInterface}} - {{- $serverInterfaceName = .ServiceName}}{{/* when streamx is enabled and there are streaming methods, refer to Server Interface defined in service/server.go */}} - {{- end}} {{- $mode := ""}} {{- if $streamingUnary -}} {{- $mode = "serviceinfo.StreamingUnary" }} {{- $handlerFunc = "InvokeUnaryHandler" }} {{- else if $serverSide -}} {{- $mode = "serviceinfo.StreamingServer" }} {{- $handlerFunc = "InvokeServerStreamHandler" }} @@ -233,8 +228,8 @@ func {{LowerFirst .Name}}Handler(ctx context.Context, handler interface{}, arg, {{- end}} {{if gt .ArgsLength 0}}realArg := {{else}}_ = {{end}}arg.(*{{if not .GenArgResultStruct}}{{.PkgRefName}}.{{end}}{{.ArgStructName}}) {{if or (not .Void) .Exceptions}}realResult := result.(*{{if not .GenArgResultStruct}}{{.PkgRefName}}.{{end}}{{.ResStructName}}){{end}} - {{if .Void}}err := handler.({{$serverInterfaceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) - {{else}}success, err := handler.({{$serverInterfaceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) + {{if .Void}}err := handler.({{.PkgRefName}}.{{.ServiceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) + {{else}}success, err := handler.({{.PkgRefName}}.{{.ServiceName}}).{{.Name}}(ctx{{range .Args}}, realArg.{{.Name}}{{end}}) {{end -}} if err != nil { {{- if $HandlerReturnKeepResp }} From c4ddb7c000404b71e0b62bf306ca4ac73ed1fccb Mon Sep 17 00:00:00 2001 From: Yi Duan Date: Fri, 14 Feb 2025 19:39:27 +0800 Subject: [PATCH 30/35] feat: update `dynamicgo` to remove build constraits (#1701) --- go.mod | 2 +- go.sum | 4 +- pkg/generic/thrift/http.go | 58 +++++++++++ pkg/generic/thrift/http_amd64.go | 90 ----------------- pkg/generic/thrift/http_fallback.go | 32 ------ pkg/generic/thrift/json.go | 112 +++++++++++++++++++++ pkg/generic/thrift/json_amd64.go | 147 ---------------------------- pkg/generic/thrift/json_fallback.go | 33 ------- 8 files changed, 173 insertions(+), 305 deletions(-) delete mode 100644 pkg/generic/thrift/http_amd64.go delete mode 100644 pkg/generic/thrift/http_fallback.go delete mode 100644 pkg/generic/thrift/json_amd64.go delete mode 100644 pkg/generic/thrift/json_fallback.go diff --git a/go.mod b/go.mod index cef81f69e7..eb34c57d66 100644 --- a/go.mod +++ b/go.mod @@ -6,7 +6,7 @@ require ( github.com/bytedance/gopkg v0.1.1 github.com/bytedance/sonic v1.12.7 github.com/cloudwego/configmanager v0.2.2 - github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71 + github.com/cloudwego/dynamicgo v0.5.2 github.com/cloudwego/fastpb v0.0.5 github.com/cloudwego/frugal v0.2.3 github.com/cloudwego/gopkg v0.1.4-0.20241217093255-8980b14172b7 diff --git a/go.sum b/go.sum index 7c0756ee0b..ef1843e085 100644 --- a/go.sum +++ b/go.sum @@ -15,8 +15,8 @@ github.com/cloudwego/base64x v0.1.4 h1:jwCgWpFanWmN8xoIUHa2rtzmkd5J2plF/dnLS6Xd/ github.com/cloudwego/base64x v0.1.4/go.mod h1:0zlkT4Wn5C6NdauXdJRhSKRlJvmclQ1hhJgA0rcu/8w= github.com/cloudwego/configmanager v0.2.2 h1:sVrJB8gWYTlPV2OS3wcgJSO9F2/9Zbkmcm1Z7jempOU= github.com/cloudwego/configmanager v0.2.2/go.mod h1:ppiyU+5TPLonE8qMVi/pFQk2eL3Q4P7d4hbiNJn6jwI= -github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71 h1:J57+W8YYGJy0MCLk/yzuLehiQmKpOoQng+OBF/5204o= -github.com/cloudwego/dynamicgo v0.5.1-0.20250115031329-d58b94fc7d71/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= +github.com/cloudwego/dynamicgo v0.5.2 h1:hw4AUvaQP49TOI6hqIhyDd4N1nbaKTH3vOOgiaEftyU= +github.com/cloudwego/dynamicgo v0.5.2/go.mod h1:DknfxjIMuGvXow409bS/AWycXONdc02HECBL0qpNqTY= github.com/cloudwego/fastpb v0.0.5 h1:vYnBPsfbAtU5TVz5+f9UTlmSCixG9F9vRwaqE0mZPZU= github.com/cloudwego/fastpb v0.0.5/go.mod h1:Bho7aAKBUtT9RPD2cNVkTdx4yQumfSv3If7wYnm1izk= github.com/cloudwego/frugal v0.2.3 h1:t1hhhAi8lXcx7Ncs4PR1pSZ90vlDU1cy5K2btDMFpoA= diff --git a/pkg/generic/thrift/http.go b/pkg/generic/thrift/http.go index 5bb0d9e8a2..ba65284a56 100644 --- a/pkg/generic/thrift/http.go +++ b/pkg/generic/thrift/http.go @@ -19,12 +19,16 @@ package thrift import ( "context" "fmt" + "unsafe" "github.com/bytedance/gopkg/lang/dirtmake" + "github.com/bytedance/gopkg/lang/mcache" "github.com/bytedance/sonic" "github.com/cloudwego/dynamicgo/conv" + "github.com/cloudwego/dynamicgo/conv/j2t" "github.com/cloudwego/dynamicgo/conv/t2j" dthrift "github.com/cloudwego/dynamicgo/thrift" + dbase "github.com/cloudwego/dynamicgo/thrift/base" "github.com/cloudwego/gopkg/bufiox" "github.com/cloudwego/gopkg/protocol/thrift" "github.com/cloudwego/gopkg/protocol/thrift/base" @@ -202,3 +206,57 @@ func (r *ReadHTTPResponse) originalRead(ctx context.Context, method string, in b } return resp, err } + +// Write ... +func (w *WriteHTTPRequest) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { + // fallback logic + if !w.dynamicgoEnabled { + return w.originalWrite(ctx, out, msg, requestBase) + } + + binaryWriter := thrift.NewBufferWriter(out) + defer binaryWriter.Recycle() + + // dynamicgo logic + req := msg.(*descriptor.HTTPRequest) + + fnDsc := w.svc.DynamicGoDsc.Functions()[method] + if fnDsc == nil { + return fmt.Errorf("missing method: %s in service: %s in dynamicgo", method, w.svc.DynamicGoDsc.Name()) + } + dynamicgoTypeDsc := fnDsc.Request() + + var cv j2t.BinaryConv + if !fnDsc.HasRequestBase() { + requestBase = nil + } + if requestBase != nil { + base := (*dbase.Base)(unsafe.Pointer(requestBase)) + ctx = context.WithValue(ctx, conv.CtxKeyThriftReqBase, base) + cv = j2t.NewBinaryConv(w.convOptsWithThriftBase) + } else { + cv = j2t.NewBinaryConv(w.convOpts) + } + + ctx = context.WithValue(ctx, conv.CtxKeyHTTPRequest, req) + body := req.GetBody() + dbuf := mcache.Malloc(len(body))[0:0] + defer mcache.Free(dbuf) + + for _, field := range dynamicgoTypeDsc.Struct().Fields() { + if err := binaryWriter.WriteFieldBegin(thrift.TType(field.Type().Type()), int16(field.ID())); err != nil { + return err + } + // json []byte to thrift []byte + if err := cv.DoInto(ctx, field.Type(), body, &dbuf); err != nil { + return err + } + if wb, err := out.Malloc(len(dbuf)); err != nil { + return err + } else { + copy(wb, dbuf) + } + dbuf = dbuf[:0] + } + return binaryWriter.WriteFieldStop() +} diff --git a/pkg/generic/thrift/http_amd64.go b/pkg/generic/thrift/http_amd64.go deleted file mode 100644 index 2911419e3d..0000000000 --- a/pkg/generic/thrift/http_amd64.go +++ /dev/null @@ -1,90 +0,0 @@ -//go:build amd64 -// +build amd64 - -/* - * Copyright 2023 CloudWeGo Authors - * - * 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 thrift - -import ( - "context" - "fmt" - "unsafe" - - "github.com/bytedance/gopkg/lang/mcache" - "github.com/cloudwego/dynamicgo/conv" - "github.com/cloudwego/dynamicgo/conv/j2t" - dbase "github.com/cloudwego/dynamicgo/thrift/base" - "github.com/cloudwego/gopkg/bufiox" - "github.com/cloudwego/gopkg/protocol/thrift" - "github.com/cloudwego/gopkg/protocol/thrift/base" - - "github.com/cloudwego/kitex/pkg/generic/descriptor" -) - -// Write ... -func (w *WriteHTTPRequest) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { - // fallback logic - if !w.dynamicgoEnabled { - return w.originalWrite(ctx, out, msg, requestBase) - } - - binaryWriter := thrift.NewBufferWriter(out) - defer binaryWriter.Recycle() - - // dynamicgo logic - req := msg.(*descriptor.HTTPRequest) - - fnDsc := w.svc.DynamicGoDsc.Functions()[method] - if fnDsc == nil { - return fmt.Errorf("missing method: %s in service: %s in dynamicgo", method, w.svc.DynamicGoDsc.Name()) - } - dynamicgoTypeDsc := fnDsc.Request() - - var cv j2t.BinaryConv - if !fnDsc.HasRequestBase() { - requestBase = nil - } - if requestBase != nil { - base := (*dbase.Base)(unsafe.Pointer(requestBase)) - ctx = context.WithValue(ctx, conv.CtxKeyThriftReqBase, base) - cv = j2t.NewBinaryConv(w.convOptsWithThriftBase) - } else { - cv = j2t.NewBinaryConv(w.convOpts) - } - - ctx = context.WithValue(ctx, conv.CtxKeyHTTPRequest, req) - body := req.GetBody() - dbuf := mcache.Malloc(len(body))[0:0] - defer mcache.Free(dbuf) - - for _, field := range dynamicgoTypeDsc.Struct().Fields() { - if err := binaryWriter.WriteFieldBegin(thrift.TType(field.Type().Type()), int16(field.ID())); err != nil { - return err - } - // json []byte to thrift []byte - if err := cv.DoInto(ctx, field.Type(), body, &dbuf); err != nil { - return err - } - if wb, err := out.Malloc(len(dbuf)); err != nil { - return err - } else { - copy(wb, dbuf) - } - dbuf = dbuf[:0] - } - return binaryWriter.WriteFieldStop() -} diff --git a/pkg/generic/thrift/http_fallback.go b/pkg/generic/thrift/http_fallback.go deleted file mode 100644 index b182b31630..0000000000 --- a/pkg/generic/thrift/http_fallback.go +++ /dev/null @@ -1,32 +0,0 @@ -//go:build !amd64 -// +build !amd64 - -/* - * Copyright 2023 CloudWeGo Authors - * - * 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 thrift - -import ( - "context" - - "github.com/cloudwego/gopkg/bufiox" - "github.com/cloudwego/gopkg/protocol/thrift/base" -) - -// Write ... -func (w *WriteHTTPRequest) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { - return w.originalWrite(ctx, out, msg, requestBase) -} diff --git a/pkg/generic/thrift/json.go b/pkg/generic/thrift/json.go index d6a6807d51..afd84e3660 100644 --- a/pkg/generic/thrift/json.go +++ b/pkg/generic/thrift/json.go @@ -20,11 +20,15 @@ import ( "context" "fmt" "strconv" + "unsafe" "github.com/bytedance/gopkg/lang/dirtmake" + "github.com/bytedance/gopkg/lang/mcache" "github.com/cloudwego/dynamicgo/conv" + "github.com/cloudwego/dynamicgo/conv/j2t" "github.com/cloudwego/dynamicgo/conv/t2j" dthrift "github.com/cloudwego/dynamicgo/thrift" + dbase "github.com/cloudwego/dynamicgo/thrift/base" "github.com/cloudwego/gopkg/bufiox" "github.com/cloudwego/gopkg/protocol/thrift" "github.com/cloudwego/gopkg/protocol/thrift/base" @@ -295,3 +299,111 @@ func structReader(ctx context.Context, typeDesc *descriptor.TypeDescriptor, opt } return string(respNode), nil } + +// Write write json string to out thrift.TProtocol +func (m *WriteJSON) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { + // fallback logic + if !m.dynamicgoEnabled { + return m.originalWrite(ctx, out, msg, method, isClient, requestBase) + } + + // dynamicgo logic + fnDsc := m.svcDsc.DynamicGoDsc.Functions()[method] + if fnDsc == nil { + return fmt.Errorf("missing method: %s in service: %s in dynamicgo", method, m.svcDsc.DynamicGoDsc.Name()) + } + dynamicgoTypeDsc := fnDsc.Request() + if !isClient { + dynamicgoTypeDsc = fnDsc.Response() + } + hasRequestBase := fnDsc.HasRequestBase() && isClient + + var cv j2t.BinaryConv + if !hasRequestBase { + requestBase = nil + } + if requestBase != nil { + base := (*dbase.Base)(unsafe.Pointer(requestBase)) + ctx = context.WithValue(ctx, conv.CtxKeyThriftReqBase, base) + cv = j2t.NewBinaryConv(m.convOptsWithThriftBase) + } else { + cv = j2t.NewBinaryConv(m.convOpts) + } + + // msg is void or nil + if _, ok := msg.(descriptor.Void); ok || msg == nil { + return writeFields(ctx, out, dynamicgoTypeDsc, nil, nil, isClient) + } + + // msg is string + s, ok := msg.(string) + if !ok { + return perrors.NewProtocolErrorWithType(perrors.InvalidData, "decode msg failed, is not string") + } + transBuff := utils.StringToSliceByte(s) + + if fnDsc.IsWithoutWrapping() { + return writeUnwrappedFields(ctx, out, dynamicgoTypeDsc, &cv, transBuff) + } else { + return writeFields(ctx, out, dynamicgoTypeDsc, &cv, transBuff, isClient) + } +} + +type MsgType int + +func writeFields(ctx context.Context, out bufiox.Writer, dynamicgoTypeDsc *dthrift.TypeDescriptor, cv *j2t.BinaryConv, transBuff []byte, isClient bool) error { + dbuf := mcache.Malloc(len(transBuff))[0:0] + defer mcache.Free(dbuf) + + bw := thrift.NewBufferWriter(out) + defer bw.Recycle() + for _, field := range dynamicgoTypeDsc.Struct().Fields() { + // Exception field + if !isClient && field.ID() != 0 { + // generic server ignore the exception, because no description for exception + // generic handler just return error + continue + } + + if err := bw.WriteFieldBegin(thrift.TType(field.Type().Type()), int16(field.ID())); err != nil { + return err + } + // if the field type is void, break + if field.Type().Type() == dthrift.VOID { + if err := bw.WriteFieldStop(); err != nil { + return err + } + break + } else { + // encode using dynamicgo + // json []byte to thrift []byte + if err := cv.DoInto(ctx, field.Type(), transBuff, &dbuf); err != nil { + return err + } + if wb, err := out.Malloc(len(dbuf)); err != nil { + return err + } else { + copy(wb, dbuf) + } + dbuf = dbuf[:0] + } + } + return bw.WriteFieldStop() +} + +func writeUnwrappedFields(ctx context.Context, out bufiox.Writer, dynamicgoTypeDsc *dthrift.TypeDescriptor, cv *j2t.BinaryConv, transBuff []byte) error { + dbuf := mcache.Malloc(len(transBuff))[0:0] + defer mcache.Free(dbuf) + + if err := cv.DoInto(ctx, dynamicgoTypeDsc, transBuff, &dbuf); err != nil { + return err + } + + if wb, err := out.Malloc(len(dbuf)); err != nil { + return err + } else { + copy(wb, dbuf) + } + dbuf = dbuf[:0] + return nil +} diff --git a/pkg/generic/thrift/json_amd64.go b/pkg/generic/thrift/json_amd64.go deleted file mode 100644 index 0e434af567..0000000000 --- a/pkg/generic/thrift/json_amd64.go +++ /dev/null @@ -1,147 +0,0 @@ -//go:build amd64 -// +build amd64 - -/* - * Copyright 2023 CloudWeGo Authors - * - * 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 thrift - -import ( - "context" - "fmt" - "unsafe" - - "github.com/bytedance/gopkg/lang/mcache" - "github.com/cloudwego/dynamicgo/conv" - "github.com/cloudwego/dynamicgo/conv/j2t" - dthrift "github.com/cloudwego/dynamicgo/thrift" - dbase "github.com/cloudwego/dynamicgo/thrift/base" - "github.com/cloudwego/gopkg/bufiox" - "github.com/cloudwego/gopkg/protocol/thrift" - "github.com/cloudwego/gopkg/protocol/thrift/base" - - "github.com/cloudwego/kitex/pkg/generic/descriptor" - "github.com/cloudwego/kitex/pkg/remote/codec/perrors" - "github.com/cloudwego/kitex/pkg/utils" -) - -// Write write json string to out thrift.TProtocol -func (m *WriteJSON) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { - // fallback logic - if !m.dynamicgoEnabled { - return m.originalWrite(ctx, out, msg, method, isClient, requestBase) - } - - // dynamicgo logic - fnDsc := m.svcDsc.DynamicGoDsc.Functions()[method] - if fnDsc == nil { - return fmt.Errorf("missing method: %s in service: %s in dynamicgo", method, m.svcDsc.DynamicGoDsc.Name()) - } - dynamicgoTypeDsc := fnDsc.Request() - if !isClient { - dynamicgoTypeDsc = fnDsc.Response() - } - hasRequestBase := fnDsc.HasRequestBase() && isClient - - var cv j2t.BinaryConv - if !hasRequestBase { - requestBase = nil - } - if requestBase != nil { - base := (*dbase.Base)(unsafe.Pointer(requestBase)) - ctx = context.WithValue(ctx, conv.CtxKeyThriftReqBase, base) - cv = j2t.NewBinaryConv(m.convOptsWithThriftBase) - } else { - cv = j2t.NewBinaryConv(m.convOpts) - } - - // msg is void or nil - if _, ok := msg.(descriptor.Void); ok || msg == nil { - return writeFields(ctx, out, dynamicgoTypeDsc, nil, nil, isClient) - } - - // msg is string - s, ok := msg.(string) - if !ok { - return perrors.NewProtocolErrorWithType(perrors.InvalidData, "decode msg failed, is not string") - } - transBuff := utils.StringToSliceByte(s) - - if fnDsc.IsWithoutWrapping() { - return writeUnwrappedFields(ctx, out, dynamicgoTypeDsc, &cv, transBuff) - } else { - return writeFields(ctx, out, dynamicgoTypeDsc, &cv, transBuff, isClient) - } -} - -type MsgType int - -func writeFields(ctx context.Context, out bufiox.Writer, dynamicgoTypeDsc *dthrift.TypeDescriptor, cv *j2t.BinaryConv, transBuff []byte, isClient bool) error { - dbuf := mcache.Malloc(len(transBuff))[0:0] - defer mcache.Free(dbuf) - - bw := thrift.NewBufferWriter(out) - defer bw.Recycle() - for _, field := range dynamicgoTypeDsc.Struct().Fields() { - // Exception field - if !isClient && field.ID() != 0 { - // generic server ignore the exception, because no description for exception - // generic handler just return error - continue - } - - if err := bw.WriteFieldBegin(thrift.TType(field.Type().Type()), int16(field.ID())); err != nil { - return err - } - // if the field type is void, break - if field.Type().Type() == dthrift.VOID { - if err := bw.WriteFieldStop(); err != nil { - return err - } - break - } else { - // encode using dynamicgo - // json []byte to thrift []byte - if err := cv.DoInto(ctx, field.Type(), transBuff, &dbuf); err != nil { - return err - } - if wb, err := out.Malloc(len(dbuf)); err != nil { - return err - } else { - copy(wb, dbuf) - } - dbuf = dbuf[:0] - } - } - return bw.WriteFieldStop() -} - -func writeUnwrappedFields(ctx context.Context, out bufiox.Writer, dynamicgoTypeDsc *dthrift.TypeDescriptor, cv *j2t.BinaryConv, transBuff []byte) error { - dbuf := mcache.Malloc(len(transBuff))[0:0] - defer mcache.Free(dbuf) - - if err := cv.DoInto(ctx, dynamicgoTypeDsc, transBuff, &dbuf); err != nil { - return err - } - - if wb, err := out.Malloc(len(dbuf)); err != nil { - return err - } else { - copy(wb, dbuf) - } - dbuf = dbuf[:0] - return nil -} diff --git a/pkg/generic/thrift/json_fallback.go b/pkg/generic/thrift/json_fallback.go deleted file mode 100644 index da214400dd..0000000000 --- a/pkg/generic/thrift/json_fallback.go +++ /dev/null @@ -1,33 +0,0 @@ -//go:build !amd64 -// +build !amd64 - -/* - * Copyright 2023 CloudWeGo Authors - * - * 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 thrift - -import ( - "context" - - "github.com/cloudwego/gopkg/bufiox" - "github.com/cloudwego/gopkg/protocol/thrift/base" -) - -// Write write json string to out thrift.TProtocol -func (m *WriteJSON) Write(ctx context.Context, out bufiox.Writer, msg interface{}, method string, isClient bool, requestBase *base.Base) error { - err := m.originalWrite(ctx, out, msg, method, isClient, requestBase) - return err -} From 26a5ec9652b2118b40ea87b0632c7f8c51771238 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Mon, 17 Feb 2025 10:56:29 +0800 Subject: [PATCH 31/35] fix(tool): refer each ServiceInfo directly when -use and -tpl multiple_services are set (#1688) --- tool/internal_pkg/generator/generator.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/tool/internal_pkg/generator/generator.go b/tool/internal_pkg/generator/generator.go index ae686f20e1..b6e78cf1b6 100644 --- a/tool/internal_pkg/generator/generator.go +++ b/tool/internal_pkg/generator/generator.go @@ -654,11 +654,11 @@ func (g *generator) setImports(name string, pkg *PackageInfo) { case MainFileName: pkg.AddImport("log", "log") if !g.Config.IsUsingMultipleServicesTpl() { - pkg.AddImport(pkg.PkgRefName, util.JoinPath(pkg.ImportPath, strings.ToLower(pkg.ServiceName))) + pkg.AddImport(pkg.PkgInfo.PkgRefName, util.JoinPath(pkg.PkgInfo.ImportPath, strings.ToLower(pkg.ServiceInfo.ServiceName))) } else { pkg.AddImports("server") for _, svc := range pkg.Services { - pkg.AddImport(svc.RefName, util.JoinPath(pkg.ImportPath, strings.ToLower(svc.ServiceName))) + pkg.AddImport(svc.RefName, util.JoinPath(svc.PkgInfo.ImportPath, strings.ToLower(svc.ServiceName))) } } } From 9a261f994723d8156fc121891c8d17785dffb667 Mon Sep 17 00:00:00 2001 From: Kyle Xiao Date: Mon, 17 Feb 2025 16:50:49 +0800 Subject: [PATCH 32/35] ci: use flag for codecov (#1704) --- .github/workflows/pr-check.yml | 1 + .github/workflows/tests.yml | 7 ++++++ .golangci.yaml | 44 +++++++++++++--------------------- 3 files changed, 25 insertions(+), 27 deletions(-) diff --git a/.github/workflows/pr-check.yml b/.github/workflows/pr-check.yml index cb4c24ff01..0117f92a4d 100644 --- a/.github/workflows/pr-check.yml +++ b/.github/workflows/pr-check.yml @@ -34,3 +34,4 @@ jobs: uses: golangci/golangci-lint-action@v6 with: version: latest + only-new-issues: true diff --git a/.github/workflows/tests.yml b/.github/workflows/tests.yml index f968144692..c47dd8b447 100644 --- a/.github/workflows/tests.yml +++ b/.github/workflows/tests.yml @@ -20,6 +20,11 @@ jobs: cd kitex-tests ./run.sh ${{github.workspace}} cd ${{github.workspace}} + - name: Upload coverage to Codecov # coverage generated by run.sh + uses: codecov/codecov-action@v5 + with: + flags: integration + benchmark-test: runs-on: ubuntu-latest @@ -119,3 +124,5 @@ jobs: run: go test -race -coverprofile=coverage.out -covermode=atomic ./... - name: Upload coverage to Codecov uses: codecov/codecov-action@v5 + with: + flags: unit diff --git a/.golangci.yaml b/.golangci.yaml index 899970a06d..61920fe82c 100644 --- a/.golangci.yaml +++ b/.golangci.yaml @@ -1,16 +1,19 @@ # Options for analysis running. run: - # include `vendor` `third_party` `testdata` `examples` `Godeps` `builtin` - skip-dirs-use-default: true - skip-dirs: - - kitex_gen - skip-files: - - ".*\\.mock\\.go$" -# output configuration options -output: - # Format: colored-line-number|line-number|json|tab|checkstyle|code-climate|junit-xml|github-actions - format: colored-line-number -# All available settings of specific linters. + timeout: 3m + +linters: # https://golangci-lint.run/usage/linters/ + disable-all: true + enable: + - gosimple + - govet + - ineffassign + - staticcheck + - unused + - unconvert + - goimports + - gofumpt + # Refer to https://golangci-lint.run/usage/linters linters-settings: gofumpt: @@ -21,21 +24,8 @@ linters-settings: # Put imports beginning with prefix after 3rd-party packages. # It's a comma-separated list of prefixes. local-prefixes: github.com/cloudwego/kitex - govet: - # Disable analyzers by name. - # Run `go tool vet help` to see all analyzers. - disable: - - stdmethods -linters: - enable: - - gofumpt - - goimports - - gofmt - disable: - - errcheck - - typecheck - - deadcode - - varcheck - - staticcheck + issues: exclude-use-default: true + exclude-dirs: + - kitex_gen From 08801429de1924d2ac4d5e0c24702050b3862406 Mon Sep 17 00:00:00 2001 From: QihengZhou Date: Mon, 17 Feb 2025 17:16:36 +0800 Subject: [PATCH 33/35] Merge pull request #1690 from ppzqh/fix/kitexutil_addr fix(kitexutil): check if address is nil when GetCallerAddress to prevent panic --- pkg/utils/kitexutil/kitexutil.go | 4 +-- pkg/utils/kitexutil/kitexutil_test.go | 45 ++++++++++++++------------- 2 files changed, 26 insertions(+), 23 deletions(-) diff --git a/pkg/utils/kitexutil/kitexutil.go b/pkg/utils/kitexutil/kitexutil.go index b8fba7e3a5..faf5c17389 100644 --- a/pkg/utils/kitexutil/kitexutil.go +++ b/pkg/utils/kitexutil/kitexutil.go @@ -92,7 +92,7 @@ func GetCallerAddr(ctx context.Context) (net.Addr, bool) { defer func() { recover() }() ri := rpcinfo.GetRPCInfo(ctx) - if ri == nil { + if ri == nil || ri.From() == nil || ri.From().Address() == nil { return nil, false } return ri.From().Address(), true @@ -104,7 +104,7 @@ func GetCallerIP(ctx context.Context) (string, bool) { defer func() { recover() }() ri := rpcinfo.GetRPCInfo(ctx) - if ri == nil { + if ri == nil || ri.From() == nil || ri.From().Address() == nil { return "", false } addrStr := ri.From().Address().String() diff --git a/pkg/utils/kitexutil/kitexutil_test.go b/pkg/utils/kitexutil/kitexutil_test.go index 58b9b858d0..76745dce0a 100644 --- a/pkg/utils/kitexutil/kitexutil_test.go +++ b/pkg/utils/kitexutil/kitexutil_test.go @@ -31,20 +31,20 @@ import ( ) var ( - testRi rpcinfo.RPCInfo - testCtx context.Context - panicCtx context.Context - caller = "kitexutil.from.service" - callee = "kitexutil.to.service" - idlServiceName = "MockService" - fromAddr = utils.NewNetAddr("test", "127.0.0.1:12345") - fromMethod = "from_method" - method = "method" - tp = transport.TTHeader + testRi rpcinfo.RPCInfo + testCtx context.Context + panicCtx context.Context + testCaller = "kitexutil.from.service" + testCallee = "kitexutil.to.service" + testIdlServiceName = "MockService" + testFromAddr = utils.NewNetAddr("test", "127.0.0.1:12345") + testFromMethod = "from_method" + testMethod = "testMethod" + testTp = transport.TTHeader ) func TestMain(m *testing.M) { - testRi = buildRPCInfo() + testRi = buildRPCInfo(testCaller, testFromMethod, testCallee, testMethod, testIdlServiceName, testFromAddr, testTp) testCtx = context.Background() testCtx = rpcinfo.NewCtxWithRPCInfo(testCtx, testRi) @@ -63,7 +63,7 @@ func TestGetCaller(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: caller, want1: true}, + {name: "Success", args: args{testCtx}, want: testCaller, want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: "", want1: false}, } @@ -90,7 +90,7 @@ func TestGetCallee(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: callee, want1: true}, + {name: "Success", args: args{testCtx}, want: testCallee, want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: "", want1: false}, } @@ -108,6 +108,8 @@ func TestGetCallee(t *testing.T) { } func TestGetCallerAddr(t *testing.T) { + riWithoutAddress := buildRPCInfo(testCaller, testFromMethod, testCallee, testMethod, testIdlServiceName, nil, testTp) + ctxWithoutAddress := rpcinfo.NewCtxWithRPCInfo(context.Background(), riWithoutAddress) type args struct { ctx context.Context } @@ -117,8 +119,9 @@ func TestGetCallerAddr(t *testing.T) { want net.Addr want1 bool }{ - {name: "Success", args: args{testCtx}, want: fromAddr, want1: true}, - {name: "Failure", args: args{context.Background()}, want: nil, want1: false}, + {name: "Success", args: args{testCtx}, want: testFromAddr, want1: true}, + {name: "Failure: nil rpcinfo", args: args{context.Background()}, want: nil, want1: false}, + {name: "Failure: nil address", args: args{ctxWithoutAddress}, want: nil, want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: nil, want1: false}, } for _, tt := range tests { @@ -139,7 +142,7 @@ func TestGetCallerIP(t *testing.T) { test.Assert(t, ok) test.Assert(t, ip == "127.0.0.1", ip) - ri := buildRPCInfo() + ri := buildRPCInfo(testCaller, testFromMethod, testCallee, testMethod, testIdlServiceName, testFromAddr, testTp) rpcinfo.AsMutableEndpointInfo(ri.From()).SetAddress(utils.NewNetAddr("test", "127.0.0.1")) ip, ok = GetCallerIP(rpcinfo.NewCtxWithRPCInfo(context.Background(), ri)) test.Assert(t, ok) @@ -169,7 +172,7 @@ func TestGetMethod(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: method, want1: true}, + {name: "Success", args: args{testCtx}, want: testMethod, want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, } for _, tt := range tests { @@ -195,7 +198,7 @@ func TestGetCallerHandlerMethod(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: fromMethod, want1: true}, + {name: "Success", args: args{testCtx}, want: testFromMethod, want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: "", want1: false}, } @@ -222,7 +225,7 @@ func TestGetIDLServiceName(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: idlServiceName, want1: true}, + {name: "Success", args: args{testCtx}, want: testIdlServiceName, want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: "", want1: false}, } @@ -275,7 +278,7 @@ func TestGetCtxTransportProtocol(t *testing.T) { want string want1 bool }{ - {name: "Success", args: args{testCtx}, want: tp.String(), want1: true}, + {name: "Success", args: args{testCtx}, want: testTp.String(), want1: true}, {name: "Failure", args: args{context.Background()}, want: "", want1: false}, {name: "Panic recovered", args: args{panicCtx}, want: "", want1: false}, } @@ -340,7 +343,7 @@ func TestGetRealResponse(t *testing.T) { } } -func buildRPCInfo() rpcinfo.RPCInfo { +func buildRPCInfo(caller, fromMethod, callee, method, idlServiceName string, fromAddr net.Addr, tp transport.Protocol) rpcinfo.RPCInfo { from := rpcinfo.NewEndpointInfo(caller, fromMethod, fromAddr, nil) to := rpcinfo.NewEndpointInfo(callee, method, nil, nil) ink := rpcinfo.NewInvocation(idlServiceName, method) From 276258ffee7b82ae622da7996cc82e5ab1136000 Mon Sep 17 00:00:00 2001 From: Scout Wang Date: Tue, 18 Feb 2025 10:55:10 +0800 Subject: [PATCH 34/35] fix(gRPC): golang CI Linter failure on http2Client (#1706) --- pkg/remote/trans/nphttp2/grpc/http2_client.go | 2 +- pkg/streamx/provider/ttstream/stream_reader_test.go | 9 ++++----- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/pkg/remote/trans/nphttp2/grpc/http2_client.go b/pkg/remote/trans/nphttp2/grpc/http2_client.go index 37ac2f32bc..5f650215c0 100644 --- a/pkg/remote/trans/nphttp2/grpc/http2_client.go +++ b/pkg/remote/trans/nphttp2/grpc/http2_client.go @@ -292,7 +292,7 @@ type closeStreamTask struct { toCloseStreams []*Stream } -func (task closeStreamTask) Tick() { +func (task *closeStreamTask) Tick() { trans := task.t trans.mu.Lock() for _, stream := range trans.activeStreams { diff --git a/pkg/streamx/provider/ttstream/stream_reader_test.go b/pkg/streamx/provider/ttstream/stream_reader_test.go index b6d831ac85..2d4c651b04 100644 --- a/pkg/streamx/provider/ttstream/stream_reader_test.go +++ b/pkg/streamx/provider/ttstream/stream_reader_test.go @@ -19,7 +19,6 @@ package ttstream import ( "context" "errors" - "sync/atomic" "testing" "time" @@ -33,20 +32,20 @@ func TestStreamReader(t *testing.T) { // basic IOs sio := newStreamReader() - var done int32 + doneCh := make(chan struct{}) go func() { for i := 0; i < round; i++ { sio.input(ctx, msg) } - atomic.StoreInt32(&done, 1) + close(doneCh) }() for i := 0; i < round; i++ { payload, err := sio.output(ctx) test.Assert(t, err == nil, err) test.DeepEqual(t, msg, payload) } - test.Assert(t, atomic.LoadInt32(&done) == int32(1)) - + // wait for goroutine exited + <-doneCh // exception IOs sio.input(ctx, msg) targetErr := errors.New("test") From 89ff9cf4ad3fae25cadfff4f3128b64dd7f1bba8 Mon Sep 17 00:00:00 2001 From: alice <90381261+alice-yyds@users.noreply.github.com> Date: Mon, 17 Feb 2025 17:34:31 +0800 Subject: [PATCH 35/35] chore: update version v0.12.2 --- version.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/version.go b/version.go index a72c64cb21..2c80c6441f 100644 --- a/version.go +++ b/version.go @@ -19,5 +19,5 @@ package kitex // Name and Version info of this framework, used for statistics and debug const ( Name = "Kitex" - Version = "v0.12.1" + Version = "v0.12.2" )