From c625326208eb1616c9d6ba6ed9611b61d6e847a1 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 15:56:23 +0530 Subject: [PATCH 01/15] Refactoring, fix syntax errors Signed-off-by: Sreekanth --- go.mod | 4 +- go.sum | 8 +- pkg/sdkclient/sourcetransformer/client.go | 57 +++- .../sourcetransformer/client_test.go | 137 +++++++--- pkg/sdkclient/sourcetransformer/interface.go | 2 +- .../forward/applier/sourcetransformer.go | 8 +- pkg/sources/forward/data_forward.go | 67 +---- pkg/sources/forward/data_forward_test.go | 91 ++++--- pkg/sources/forward/shutdown_test.go | 12 +- pkg/sources/transformer/grpc_transformer.go | 170 ++++++------ .../transformer/grpc_transformer_test.go | 250 +++++++++++++----- pkg/udf/rpc/grpc_batch_map.go | 24 +- pkg/udf/rpc/tracker.go | 34 +-- pkg/udf/rpc/tracker_test.go | 12 +- 14 files changed, 536 insertions(+), 340 deletions(-) diff --git a/go.mod b/go.mod index f0dd236bd7..5eca92dddc 100644 --- a/go.mod +++ b/go.mod @@ -32,7 +32,7 @@ require ( github.com/montanaflynn/stats v0.0.0-20171201202039-1bf9dbcd8cbe github.com/nats-io/nats-server/v2 v2.10.17 github.com/nats-io/nats.go v1.36.0 - github.com/numaproj/numaflow-go v0.8.0 + github.com/numaproj/numaflow-go v0.8.2-0.20240916154529-f205113bba8e github.com/prometheus/client_golang v1.18.0 github.com/prometheus/client_model v0.5.0 github.com/prometheus/common v0.45.0 @@ -51,7 +51,7 @@ require ( golang.org/x/crypto v0.24.0 golang.org/x/net v0.25.0 golang.org/x/oauth2 v0.20.0 - golang.org/x/sync v0.7.0 + golang.org/x/sync v0.8.0 golang.org/x/tools v0.21.1-0.20240508182429-e35e4ccd0d2d google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 google.golang.org/grpc v1.59.0 diff --git a/go.sum b/go.sum index 1e46c54600..7dfca21927 100644 --- a/go.sum +++ b/go.sum @@ -485,8 +485,8 @@ github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDm github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/numaproj/numaflow-go v0.8.0 h1:1Pp0AMLXkmUPlvFjKeY3a9X+OLU8oN1OQWxD9jLg8Uo= -github.com/numaproj/numaflow-go v0.8.0/go.mod h1:WoMt31+h3up202zTRI8c/qe42B8UbvwLe2mJH0MAlhI= +github.com/numaproj/numaflow-go v0.8.2-0.20240916154529-f205113bba8e h1:ApgoGlSvmSo/+/P9XF/XeY7afWE0jSxAFgkeFaM6IW4= +github.com/numaproj/numaflow-go v0.8.2-0.20240916154529-f205113bba8e/go.mod h1:s4F+Xh4BKjSEdL+49sxKgJq8MQO41aVjN4Try4gsKwg= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= @@ -802,8 +802,8 @@ golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20220722155255-886fb9371eb4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.7.0 h1:YsImfSBoP9QPYL0xyKJPq0gcaJdG3rInoqxTWbfQu9M= -golang.org/x/sync v0.7.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= +golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= +golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= diff --git a/pkg/sdkclient/sourcetransformer/client.go b/pkg/sdkclient/sourcetransformer/client.go index d9d47302c0..0912ef733d 100644 --- a/pkg/sdkclient/sourcetransformer/client.go +++ b/pkg/sdkclient/sourcetransformer/client.go @@ -18,6 +18,8 @@ package sourcetransformer import ( "context" + "errors" + "io" "google.golang.org/grpc" "google.golang.org/protobuf/types/known/emptypb" @@ -64,7 +66,7 @@ func NewFromClient(c transformpb.SourceTransformClient) (Client, error) { } // CloseConn closes the grpc client connection. -func (c *client) CloseConn(ctx context.Context) error { +func (c *client) CloseConn(_ context.Context) error { if c.conn == nil { return nil } @@ -81,11 +83,54 @@ func (c *client) IsReady(ctx context.Context, in *emptypb.Empty) (bool, error) { } // SourceTransformFn SourceTransformerFn applies a function to each request element. -func (c *client) SourceTransformFn(ctx context.Context, request *transformpb.SourceTransformRequest) (*transformpb.SourceTransformResponse, error) { - transformResponse, err := c.grpcClt.SourceTransformFn(ctx, request) - err = sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) +func (c *client) SourceTransformFn(ctx context.Context, request <-chan *transformpb.SourceTransformRequest) (<-chan *transformpb.SourceTransformResponse, <-chan error) { + errCh := make(chan error) + responseCh := make(chan *transformpb.SourceTransformResponse) + stream, err := c.grpcClt.SourceTransformFn(ctx) if err != nil { - return nil, err + go func() { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) + }() + return nil, errCh } - return transformResponse, nil + + go func() { + defer close(responseCh) + for { + resp, err := stream.Recv() + if err != nil { + if errors.Is(err, io.EOF) { + errCh = nil + return + } + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) + return + } + responseCh <- resp + } + }() + + go func() { + for { + select { + case <-ctx.Done(): + return + case msg, ok := <-request: + if !ok { + err := stream.CloseSend() + if err != nil && errors.Is(err, io.EOF) { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn stream.CloseSend", err) + } + return + } + err := stream.Send(msg) + if err != nil { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn stream.Send", err) + return + } + } + } + }() + + return responseCh, errCh } diff --git a/pkg/sdkclient/sourcetransformer/client_test.go b/pkg/sdkclient/sourcetransformer/client_test.go index 27526312fd..b528b037bc 100644 --- a/pkg/sdkclient/sourcetransformer/client_test.go +++ b/pkg/sdkclient/sourcetransformer/client_test.go @@ -18,15 +18,22 @@ package sourcetransformer import ( "context" + "errors" "fmt" - "reflect" - "testing" - "github.com/golang/mock/gomock" transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" + "github.com/numaproj/numaflow-go/pkg/sourcetransformer" "github.com/stretchr/testify/assert" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/test/bufconn" "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/timestamppb" + "net" + "reflect" + "testing" + "time" ) func TestClient_IsReady(t *testing.T) { @@ -54,44 +61,100 @@ func TestClient_IsReady(t *testing.T) { assert.EqualError(t, err, "mock connection refused") } -func TestClient_SourceTransformFn(t *testing.T) { - var ctx = context.Background() +func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { + lis := bufconn.Listen(100) + t.Cleanup(func() { + _ = lis.Close() + }) - ctrl := gomock.NewController(t) - defer ctrl.Finish() + server := grpc.NewServer() + t.Cleanup(func() { + server.Stop() + }) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).Return(&transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, nil) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).Return(&transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, fmt.Errorf("mock connection refused")) + register(server) - testClient, err := NewFromClient(mockClient) - assert.NoError(t, err) - reflect.DeepEqual(testClient, &client{ - grpcClt: mockClient, + errChan := make(chan error, 1) + go func() { + // t.Fatal should only be called from the goroutine running the test + if err := server.Serve(lis); err != nil { + errChan <- err + } + }() + + dialer := func(context.Context, string) (net.Conn, error) { + return lis.Dial() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + t.Cleanup(func() { + cancel() }) - result, err := testClient.SourceTransformFn(ctx, &transformpb.SourceTransformRequest{}) - assert.Equal(t, &transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, result) - assert.NoError(t, err) + conn, err := grpc.DialContext(ctx, "", grpc.WithContextDialer(dialer), grpc.WithTransportCredentials(insecure.NewCredentials())) + t.Cleanup(func() { + _ = conn.Close() + }) + if err != nil { + t.Fatalf("Creating new gRPC client connection: %v", err) + } + + var grpcServerErr error + select { + case grpcServerErr = <-errChan: + case <-time.After(500 * time.Millisecond): + grpcServerErr = errors.New("gRPC server didn't start in 500ms") + } + if err != nil { + t.Fatalf("Failed to start gRPC server: %v", grpcServerErr) + } + + return conn +} - _, err = testClient.SourceTransformFn(ctx, &transformpb.SourceTransformRequest{}) - assert.EqualError(t, err, "NonRetryable: mock connection refused") +func TestClient_SourceTransformFn(t *testing.T) { + var testTime = time.Date(2021, 8, 15, 14, 30, 45, 100, time.Local) + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + msg := datum.Value() + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(msg, testTime).WithKeys([]string{keys[0] + "_test"})) + }), + } + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + var ctx = context.Background() + client, _ := NewFromClient(transformClient) + + reqChan := make(chan *transformpb.SourceTransformRequest, 1) + go func() { + for i := 0; i < 5; i++ { + reqChan <- &transformpb.SourceTransformRequest{ + Keys: []string{fmt.Sprintf("client_key_%d", i)}, + Value: []byte("test"), + } + } + }() + + respChan, errChan := client.SourceTransformFn(ctx, reqChan) + var results [][]*transformpb.SourceTransformResponse_Result + for i := 0; i < 5; i++ { + var resp *transformpb.SourceTransformResponse + var err error + select { + case resp = <-respChan: + case err = <-errChan: + } + assert.NoError(t, err) + results = append(results, resp.GetResults()) + } + expected := [][]*transformpb.SourceTransformResponse_Result{ + {{Keys: []string{"client_key_0_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_1_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_2_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_3_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_4_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + } + assert.ElementsMatch(t, expected, results) } diff --git a/pkg/sdkclient/sourcetransformer/interface.go b/pkg/sdkclient/sourcetransformer/interface.go index 4d8e3d8f71..6006e380c8 100644 --- a/pkg/sdkclient/sourcetransformer/interface.go +++ b/pkg/sdkclient/sourcetransformer/interface.go @@ -27,5 +27,5 @@ import ( type Client interface { CloseConn(ctx context.Context) error IsReady(ctx context.Context, in *emptypb.Empty) (bool, error) - SourceTransformFn(ctx context.Context, request *transformpb.SourceTransformRequest) (*transformpb.SourceTransformResponse, error) + SourceTransformFn(ctx context.Context, request <-chan *transformpb.SourceTransformRequest) (<-chan *transformpb.SourceTransformResponse, <-chan error) } diff --git a/pkg/sources/forward/applier/sourcetransformer.go b/pkg/sources/forward/applier/sourcetransformer.go index 795cd4c5a2..a935d511ea 100644 --- a/pkg/sources/forward/applier/sourcetransformer.go +++ b/pkg/sources/forward/applier/sourcetransformer.go @@ -25,13 +25,13 @@ import ( // SourceTransformApplier applies the source transform on the read message and gives back a new message. Any UserError will be retried here, while // InternalErr can be returned and could be retried by the callee. type SourceTransformApplier interface { - ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) + ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) } // ApplySourceTransformFunc is a function type that implements SourceTransformApplier interface. -type ApplySourceTransformFunc func(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) +type ApplySourceTransformFunc func(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) // ApplyTransform implements SourceTransformApplier interface. -func (f ApplySourceTransformFunc) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return f(ctx, message) +func (f ApplySourceTransformFunc) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + return f(ctx, messages) } diff --git a/pkg/sources/forward/data_forward.go b/pkg/sources/forward/data_forward.go index 48ff97da3a..9eddafa30d 100644 --- a/pkg/sources/forward/data_forward.go +++ b/pkg/sources/forward/data_forward.go @@ -305,34 +305,14 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // If a user-defined transformer exists, apply it if df.opts.transformer != nil { - // user-defined transformer concurrent processing request channel - transformerCh := make(chan *isb.ReadWriteMessagePair) - - // create a pool of Transformer Processors - var wg sync.WaitGroup - for i := 0; i < df.opts.transformerConcurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - df.concurrentApplyTransformer(ctx, transformerCh) - }() + for _, m := range readMessages { + // assign watermark to the message + m.Watermark = time.Time(processorWM) } concurrentTransformerProcessingStart := time.Now() - for idx, m := range readMessages { + readWriteMessagePairs = df.applyTransformer(ctx, readMessages) - // assign watermark to the message - m.Watermark = time.Time(processorWM) - readWriteMessagePairs[idx].ReadMessage = m - // send transformer processing work to the channel. Thus, the results of the transformer - // application on a read message will be stored as the corresponding writeMessage in readWriteMessagePairs - transformerCh <- &readWriteMessagePairs[idx] - } - // let the go routines know that there is no more work - close(transformerCh) - // wait till the processing is done. this will not be an infinite wait because the transformer processing will exit if - // context.Done() is closed. - wg.Wait() df.opts.logger.Debugw("concurrent applyTransformer completed", zap.Int("concurrency", df.opts.transformerConcurrency), zap.Duration("took", time.Since(concurrentTransformerProcessingStart)), @@ -661,42 +641,12 @@ func (df *DataForward) writeToBuffer(ctx context.Context, toBufferPartition isb. return writeOffsets, nil } -// concurrentApplyTransformer applies the transformer based on the request from the channel -func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessagePair <-chan *isb.ReadWriteMessagePair) { - for message := range readMessagePair { - start := time.Now() - metrics.SourceTransformerReadMessagesCount.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Inc() - - writeMessages, err := df.applyTransformer(ctx, message.ReadMessage) - metrics.SourceTransformerWriteMessagesCount.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Add(float64(len(writeMessages))) - - message.WriteMessages = append(message.WriteMessages, writeMessages...) - message.Err = err - metrics.SourceTransformerProcessingTime.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Observe(float64(time.Since(start).Microseconds())) - } -} - // applyTransformer applies the transformer and will block if there is any InternalErr. On the other hand, if this is a UserError // the skip flag is set. The ShutDown flag will only if there is an InternalErr and ForceStop has been invoked. // The UserError retry will be done on the applyTransformer. -func (df *DataForward) applyTransformer(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (df *DataForward) applyTransformer(ctx context.Context, messages []*isb.ReadMessage) []isb.ReadWriteMessagePair { for { - writeMessages, err := df.opts.transformer.ApplyTransform(ctx, readMessage) + transformResults, err := df.opts.transformer.ApplyTransform(ctx, messages) if err != nil { df.opts.logger.Errorw("Transformer.Apply error", zap.Error(err)) // TODO: implement retry with backoff etc. @@ -712,12 +662,11 @@ func (df *DataForward) applyTransformer(ctx context.Context, readMessage *isb.Re metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), }).Inc() - - return nil, err + return []isb.ReadWriteMessagePair{{Err: err}} } continue } - return writeMessages, nil + return transformResults } } diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index 25e41a9fa6..e4fb00a2e0 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -121,8 +121,16 @@ func (f myForwardTest) WhereTo(_ []string, _ []string, s string) ([]forwarder.Ve }}, nil } -func (f myForwardTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + out := make([]isb.ReadWriteMessagePair, len(messages)) + for i, msg := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", msg) + out[i] = isb.ReadWriteMessagePair{ + ReadMessage: msg, + WriteMessages: writeMsg, + } + } + return out, nil } func TestNewDataForward(t *testing.T) { @@ -856,36 +864,31 @@ func (f *mySourceForwardTestRoundRobin) WhereTo(_ []string, _ []string, s string // such that we can verify message IsLate attribute gets set to true. var testSourceNewEventTime = testSourceWatermark.Add(time.Duration(-1) * time.Minute) -func (f mySourceForwardTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return func(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { - _ = ctx - offset := readMessage.ReadOffset - payload := readMessage.Body.Payload - parentPaneInfo := readMessage.MessageInfo - - // apply source data transformer - _ = payload - // copy the payload - result := payload - // assign new event time - parentPaneInfo.EventTime = testSourceNewEventTime - var key []string - - writeMessage := isb.Message{ +func (f mySourceForwardTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + message.MessageInfo.EventTime = testSourceNewEventTime + writeMsg := isb.Message{ Header: isb.Header{ - MessageInfo: parentPaneInfo, + MessageInfo: message.MessageInfo, ID: isb.MessageID{ VertexName: "test-vertex", - Offset: offset.String(), + Offset: message.ReadOffset.String(), }, - Keys: key, + Keys: []string{}, }, Body: isb.Body{ - Payload: result, + Payload: message.Body.Payload, }, } - return []*isb.WriteMessage{{Message: writeMessage}}, nil - }(ctx, message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: []*isb.WriteMessage{&isb.WriteMessage{ + Message: writeMsg, + }}, + } + } + return results, nil } // TestSourceWatermarkPublisher is a dummy implementation of isb.SourceWatermarkPublisher interface @@ -1153,8 +1156,16 @@ func (f myForwardDropTest) WhereTo(_ []string, _ []string, s string) ([]forwarde return []forwarder.VertexBuffer{}, nil } -func (f myForwardDropTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardDropTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardToAllTest struct { @@ -1174,8 +1185,16 @@ func (f *myForwardToAllTest) WhereTo(_ []string, _ []string, s string) ([]forwar return output, nil } -func (f *myForwardToAllTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f *myForwardToAllTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardInternalErrTest struct { @@ -1188,7 +1207,7 @@ func (f myForwardInternalErrTest) WhereTo(_ []string, _ []string, s string) ([]f }}, nil } -func (f myForwardInternalErrTest) ApplyTransform(_ context.Context, _ *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (f myForwardInternalErrTest) ApplyTransform(ctx context.Context, _ []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { return nil, &udfapplier.ApplyUDFErr{ UserUDFErr: false, InternalErr: struct { @@ -1209,8 +1228,16 @@ func (f myForwardApplyWhereToErrTest) WhereTo(_ []string, _ []string, s string) }}, fmt.Errorf("whereToStep failed") } -func (f myForwardApplyWhereToErrTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardApplyWhereToErrTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardApplyTransformerErrTest struct { @@ -1223,7 +1250,7 @@ func (f myForwardApplyTransformerErrTest) WhereTo(_ []string, _ []string, s stri }}, nil } -func (f myForwardApplyTransformerErrTest) ApplyTransform(_ context.Context, _ *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (f myForwardApplyTransformerErrTest) ApplyTransform(_ context.Context, _ []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { return nil, fmt.Errorf("transformer error") } diff --git a/pkg/sources/forward/shutdown_test.go b/pkg/sources/forward/shutdown_test.go index a4ffc5e2e2..34003e729f 100644 --- a/pkg/sources/forward/shutdown_test.go +++ b/pkg/sources/forward/shutdown_test.go @@ -43,8 +43,16 @@ func (s myShutdownTest) WhereTo([]string, []string, string) ([]forwarder.VertexB return []forwarder.VertexBuffer{}, nil } -func (s myShutdownTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "", message) +func (f myShutdownTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } func (s myShutdownTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { diff --git a/pkg/sources/transformer/grpc_transformer.go b/pkg/sources/transformer/grpc_transformer.go index 14b414a348..c7f9902603 100644 --- a/pkg/sources/transformer/grpc_transformer.go +++ b/pkg/sources/transformer/grpc_transformer.go @@ -18,19 +18,17 @@ package transformer import ( "context" + "errors" "fmt" "time" v1 "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" - "google.golang.org/protobuf/types/known/emptypb" - "google.golang.org/protobuf/types/known/timestamppb" - "k8s.io/apimachinery/pkg/util/wait" - "github.com/numaproj/numaflow/pkg/isb" - sdkerr "github.com/numaproj/numaflow/pkg/sdkclient/error" "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/rpc" + "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/timestamppb" ) // GRPCBasedTransformer applies user-defined transformer over gRPC (over Unix Domain Socket) client/server where server is the transformer. @@ -75,59 +73,36 @@ func (u *GRPCBasedTransformer) CloseConn(ctx context.Context) error { return u.client.CloseConn(ctx) } -func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { - keys := readMessage.Keys - payload := readMessage.Body.Payload - offset := readMessage.ReadOffset - parentMessageInfo := readMessage.MessageInfo - var req = &v1.SourceTransformRequest{ - Keys: keys, - Value: payload, - EventTime: timestamppb.New(parentMessageInfo.EventTime), - Watermark: timestamppb.New(readMessage.Watermark), - Headers: readMessage.Headers, - } +func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + var transformResults []isb.ReadWriteMessagePair + inputChan := make(chan *v1.SourceTransformRequest) + respChan, errChan := u.client.SourceTransformFn(ctx, inputChan) - response, err := u.client.SourceTransformFn(ctx, req) - if err != nil { - udfErr, _ := sdkerr.FromError(err) - switch udfErr.ErrorKind() { - case sdkerr.Retryable: - var success bool - _ = wait.ExponentialBackoffWithContext(ctx, wait.Backoff{ - // retry every "duration * factor + [0, jitter]" interval for 5 times - Duration: 1 * time.Second, - Factor: 1, - Jitter: 0.1, - Steps: 5, - }, func(_ context.Context) (done bool, err error) { - response, err = u.client.SourceTransformFn(ctx, req) - if err != nil { - udfErr, _ = sdkerr.FromError(err) - switch udfErr.ErrorKind() { - case sdkerr.Retryable: - return false, nil - case sdkerr.NonRetryable: - return true, nil - default: - return true, nil - } - } - success = true - return true, nil - }) - if !success { - return nil, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - } + tracker := rpc.NewTracker() + + go func() { + defer close(inputChan) + for _, msg := range messages { + tracker.AddRequest(msg) + req := &v1.SourceTransformRequest{ + Keys: msg.Keys, + Value: msg.Body.Payload, + EventTime: timestamppb.New(msg.MessageInfo.EventTime), + Watermark: timestamppb.New(msg.Watermark), + Headers: msg.Headers, + Id: msg.ReadOffset.String(), } - case sdkerr.NonRetryable: - return nil, &rpc.ApplyUDFErr{ + inputChan <- req + } + }() + + messageCount := len(messages) + +loop: + for { + select { + case err := <-errChan: + err = &rpc.ApplyUDFErr{ UserUDFErr: false, Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), InternalErr: rpc.InternalErr{ @@ -135,43 +110,58 @@ func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, readMessage * MainCarDown: false, }, } - default: - return nil, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, + return nil, err + case resp, ok := <-respChan: + if !ok { + break loop + } + msgId := resp.GetId() + parentMessage, ok := tracker.GetRequest(msgId) + if !ok { + return nil, errors.New("tracker doesn't contain the message ID received from the response") + } + var taggedMessages []*isb.WriteMessage + for i, result := range resp.GetResults() { + keys := result.Keys + if result.EventTime != nil { + // Transformer supports changing event time. + parentMessage.MessageInfo.EventTime = result.EventTime.AsTime() + } + taggedMessage := &isb.WriteMessage{ + Message: isb.Message{ + Header: isb.Header{ + MessageInfo: parentMessage.MessageInfo, + ID: isb.MessageID{ + VertexName: u.vertexName, + Offset: parentMessage.ReadOffset.String(), + Index: int32(i), + }, + Keys: keys, + }, + Body: isb.Body{ + Payload: result.Value, + }, + }, + Tags: result.Tags, + } + taggedMessages = append(taggedMessages, taggedMessage) + } + responsePair := isb.ReadWriteMessagePair{ + ReadMessage: parentMessage, + WriteMessages: taggedMessages, + Err: nil, + } + transformResults = append(transformResults, responsePair) + tracker.RemoveRequest(msgId) + messageCount-- + if messageCount == 0 { + break loop } } } - taggedMessages := make([]*isb.WriteMessage, 0) - for i, result := range response.GetResults() { - keys := result.Keys - if result.EventTime != nil { - // Transformer supports changing event time. - parentMessageInfo.EventTime = result.EventTime.AsTime() - } - taggedMessage := &isb.WriteMessage{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: parentMessageInfo, - ID: isb.MessageID{ - VertexName: u.vertexName, - Offset: offset.String(), - Index: int32(i), - }, - Keys: keys, - }, - Body: isb.Body{ - Payload: result.Value, - }, - }, - Tags: result.Tags, - } - taggedMessages = append(taggedMessages, taggedMessage) + if !tracker.IsEmpty() { + return nil, errors.New("transform response for all requests were not received from UDF") } - return taggedMessages, nil + return transformResults, nil } diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 959a40bf51..86d25c147c 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -19,12 +19,17 @@ package transformer import ( "context" "encoding/json" + "errors" "fmt" + "github.com/numaproj/numaflow-go/pkg/sourcetransformer" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/test/bufconn" + "net" "testing" "time" "github.com/golang/mock/gomock" - v1 "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" + transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" "github.com/stretchr/testify/assert" "google.golang.org/grpc" @@ -35,12 +40,12 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" - "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" + sourcetransformerSdk "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/udf/rpc" ) func NewMockGRPCBasedTransformer(mockClient *transformermock.MockSourceTransformClient) *GRPCBasedTransformer { - c, _ := sourcetransformer.NewFromClient(mockClient) + c, _ := sourcetransformerSdk.NewFromClient(mockClient) return &GRPCBasedTransformer{"test-vertex", c} } @@ -49,7 +54,7 @@ func TestGRPCBasedTransformer_WaitUntilReadyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&v1.ReadyResponse{Ready: true}, nil) + mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&transformpb.ReadyResponse{Ready: true}, nil) ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() @@ -88,14 +93,14 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169600, 0)), Watermark: timestamppb.New(time.Time{}), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&v1.SourceTransformResponse{ - Results: []*v1.SourceTransformResponse_Result{ + mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ + Results: []*transformpb.SourceTransformResponse_Result{ { Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), @@ -113,7 +118,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -130,11 +135,11 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].Keys) - assert.Equal(t, req.Value, got[0].Payload) + assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) + assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) }) t.Run("test error", func(t *testing.T) { @@ -142,7 +147,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -160,7 +165,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -177,7 +182,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -194,7 +199,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -217,7 +222,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -234,7 +239,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -251,7 +256,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -270,7 +275,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -287,7 +292,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -304,15 +309,15 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169720, 0)), Watermark: timestamppb.New(time.Time{}), } mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&v1.SourceTransformResponse{ - Results: []*v1.SourceTransformResponse_Result{ + mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ + Results: []*transformpb.SourceTransformResponse_Result{ { Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), @@ -330,7 +335,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -347,11 +352,11 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].Keys) - assert.Equal(t, req.Value, got[0].Payload) + assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) + assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) }) t.Run("test error non retryable", func(t *testing.T) { @@ -359,7 +364,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -376,7 +381,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -393,7 +398,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -406,6 +411,72 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }) } +func TestGRPCBasedTransformer_ApplyWithServer_ChangePayload(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + var originalValue testutils.PayloadForTest + _ = json.Unmarshal(datum.Value(), &originalValue) + doubledValue := testutils.PayloadForTest{ + Value: originalValue.Value * 2, + Key: originalValue.Key, + } + doubledValueBytes, _ := json.Marshal(&doubledValue) + + var resultKeys []string + if originalValue.Value%2 == 0 { + resultKeys = []string{"even"} + } else { + resultKeys = []string{"odd"} + } + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(doubledValueBytes, datum.EventTime()).WithKeys(resultKeys)) + }), + } + + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) + + var count = int64(10) + readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) + messages := make([]*isb.ReadMessage, len(readMessages)) + for idx, readMessage := range readMessages { + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(context.TODO(), messages) + assert.NoError(t, err) + + var results = make([][]byte, len(readMessages)) + var resultKeys = make([][]string, len(readMessages)) + for idx, pair := range apply { + results[idx] = pair.WriteMessages[0].Payload + resultKeys[idx] = pair.WriteMessages[0].Header.Keys + } + + var expectedResults = make([][]byte, count) + var expectedKeys = make([][]string, count) + for idx, readMessage := range readMessages { + var readMessagePayload testutils.PayloadForTest + _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) + if readMessagePayload.Value%2 == 0 { + expectedKeys[idx] = []string{"even"} + } else { + expectedKeys[idx] = []string{"odd"} + } + doubledValue := testutils.PayloadForTest{ + Key: readMessagePayload.Key, + Value: readMessagePayload.Value * 2, + } + marshal, _ := json.Marshal(doubledValue) + expectedResults[idx] = marshal + } + + assert.ElementsMatch(t, expectedResults, results) + assert.Equal(t, expectedKeys, resultKeys) +} + func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { multiplyBy2 := func(body []byte) interface{} { var result testutils.PayloadForTest @@ -419,23 +490,23 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { mockClient := transformermock.NewMockSourceTransformClient(ctrl) mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *v1.SourceTransformRequest, opts ...grpc.CallOption) (*v1.SourceTransformResponse, error) { + func(_ context.Context, datum *transformpb.SourceTransformRequest, opts ...grpc.CallOption) (*transformpb.SourceTransformResponse, error) { var originalValue testutils.PayloadForTest _ = json.Unmarshal(datum.GetValue(), &originalValue) doubledValue, _ := json.Marshal(multiplyBy2(datum.GetValue()).(testutils.PayloadForTest)) - var Results []*v1.SourceTransformResponse_Result + var Results []*transformpb.SourceTransformResponse_Result if originalValue.Value%2 == 0 { - Results = append(Results, &v1.SourceTransformResponse_Result{ + Results = append(Results, &transformpb.SourceTransformResponse_Result{ Keys: []string{"even"}, Value: doubledValue, }) } else { - Results = append(Results, &v1.SourceTransformResponse_Result{ + Results = append(Results, &transformpb.SourceTransformResponse_Result{ Keys: []string{"odd"}, Value: doubledValue, }) } - datumList := &v1.SourceTransformResponse{ + datumList := &transformpb.SourceTransformResponse{ Results: Results, } return datumList, nil @@ -458,11 +529,15 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { var results = make([][]byte, len(readMessages)) var resultKeys = make([][]string, len(readMessages)) + messages := make([]*isb.ReadMessage, len(readMessages)) for idx, readMessage := range readMessages { - apply, err := u.ApplyTransform(ctx, &readMessage) - assert.NoError(t, err) - results[idx] = apply[0].Payload - resultKeys[idx] = apply[0].Header.Keys + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(ctx, messages) + assert.NoError(t, err) + for idx, pair := range apply { + results[idx] = pair.WriteMessages[0].Payload + resultKeys[idx] = pair.WriteMessages[0].Header.Keys } var expectedResults = make([][]byte, count) @@ -483,43 +558,82 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { assert.Equal(t, expectedKeys, resultKeys) } -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { - testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { + lis := bufconn.Listen(100) + t.Cleanup(func() { + _ = lis.Close() + }) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *v1.SourceTransformRequest, opts ...grpc.CallOption) (*v1.SourceTransformResponse, error) { - var Results []*v1.SourceTransformResponse_Result - Results = append(Results, &v1.SourceTransformResponse_Result{ - Keys: []string{"even"}, - Value: datum.Value, - EventTime: timestamppb.New(testEventTime), - }) - datumList := &v1.SourceTransformResponse{ - Results: Results, - } - return datumList, nil - }, - ).AnyTimes() + server := grpc.NewServer() + t.Cleanup(func() { + server.Stop() + }) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() + register(server) + + errChan := make(chan error, 1) go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") + // t.Fatal should only be called from the goroutine running the test + if err := server.Serve(lis); err != nil { + errChan <- err } }() - u := NewMockGRPCBasedTransformer(mockClient) + dialer := func(context.Context, string) (net.Conn, error) { + return lis.Dial() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + t.Cleanup(func() { + cancel() + }) + + conn, err := grpc.DialContext(ctx, "", grpc.WithContextDialer(dialer), grpc.WithTransportCredentials(insecure.NewCredentials())) + t.Cleanup(func() { + _ = conn.Close() + }) + if err != nil { + t.Fatalf("Creating new gRPC client connection: %v", err) + } + + var grpcServerErr error + select { + case grpcServerErr = <-errChan: + case <-time.After(500 * time.Millisecond): + grpcServerErr = errors.New("gRPC server didn't start in 500ms") + } + if err != nil { + t.Fatalf("Failed to start gRPC server: %v", grpcServerErr) + } + + return conn +} + +func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { + testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + msg := datum.Value() + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(msg, testEventTime).WithKeys([]string{"even"})) + }), + } + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) var count = int64(2) readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) - for _, readMessage := range readMessages { - apply, err := u.ApplyTransform(ctx, &readMessage) - assert.NoError(t, err) - assert.Equal(t, testEventTime, apply[0].EventTime) + messages := make([]*isb.ReadMessage, len(readMessages)) + for idx, readMessage := range readMessages { + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(context.TODO(), messages) + assert.NoError(t, err) + for _, pair := range apply { + assert.NoError(t, pair.Err) + assert.Equal(t, testEventTime, pair.WriteMessages[0].EventTime) } } diff --git a/pkg/udf/rpc/grpc_batch_map.go b/pkg/udf/rpc/grpc_batch_map.go index 6d6c397642..8d072a73c8 100644 --- a/pkg/udf/rpc/grpc_batch_map.go +++ b/pkg/udf/rpc/grpc_batch_map.go @@ -34,7 +34,7 @@ import ( type GRPCBasedBatchMap struct { vertexName string client batchmapper.Client - requestTracker *tracker + requestTracker *Tracker } func NewUDSgRPCBasedBatchMap(vertexName string, client batchmapper.Client) *GRPCBasedBatchMap { @@ -44,7 +44,7 @@ func NewUDSgRPCBasedBatchMap(vertexName string, client batchmapper.Client) *GRPC // requestTracker is used to store the read messages in a key, value manner where // key is the read offset and the reference to read message as the value. // Once the results are received from the UDF, we map the responses to the corresponding request - // using a lookup on this tracker. + // using a lookup on this Tracker. requestTracker: NewTracker(), } } @@ -93,18 +93,18 @@ func (u *GRPCBasedBatchMap) ApplyBatchMap(ctx context.Context, messages []*isb.R // trackerReq is used to store the read messages in a key, value manner where // key is the read offset and the reference to read message as the value. // Once the results are received from the UDF, we map the responses to the corresponding request - // using a lookup on this tracker. + // using a lookup on this Tracker. trackerReq := NewTracker() // Read routine: this goroutine iterates over the input messages and sends each // of the read messages to the grpc client after transforming it to a BatchMapRequest. // Once all messages are sent, it closes the input channel to indicate that all requests have been read. - // On creating a new request, we add it to a tracker map so that the responses on the stream + // On creating a new request, we add it to a Tracker map so that the responses on the stream // can be mapped backed to the given parent request go func() { defer close(inputChan) for _, msg := range messages { - trackerReq.addRequest(msg) + trackerReq.AddRequest(msg) inputChan <- u.parseInputRequest(msg) } }() @@ -139,14 +139,14 @@ loop: } // Get the unique request ID for which these responses are meant for. msgId := grpcResp.GetId() - // Fetch the request value for the given ID from the tracker - parentMessage, ok := trackerReq.getRequest(msgId) + // Fetch the request value for the given ID from the Tracker + parentMessage, ok := trackerReq.GetRequest(msgId) if !ok { - // this case is when the given request ID was not present in the tracker. + // this case is when the given request ID was not present in the Tracker. // This means that either the UDF added an incorrect ID // This cannot be processed further and should result in an error // Can there be another case for this? - logger.Error("Request missing from tracker, ", msgId) + logger.Error("Request missing from Tracker, ", msgId) return nil, fmt.Errorf("incorrect ID found during batch map processing") } // parse the responses received @@ -159,12 +159,12 @@ loop: Err: nil, } udfResults = append(udfResults, responsePair) - trackerReq.removeRequest(msgId) + trackerReq.RemoveRequest(msgId) } } - // check if there are elements left in the tracker. This cannot be an acceptable case as we want the + // check if there are elements left in the Tracker. This cannot be an acceptable case as we want the // UDF to send responses for all elements. - if !trackerReq.isEmpty() { + if !trackerReq.IsEmpty() { logger.Error("BatchMap response for all requests not received from UDF") return nil, fmt.Errorf("batchMap response for all requests not received from UDF") } diff --git a/pkg/udf/rpc/tracker.go b/pkg/udf/rpc/tracker.go index 60b57a7af9..0065f02eb1 100644 --- a/pkg/udf/rpc/tracker.go +++ b/pkg/udf/rpc/tracker.go @@ -6,52 +6,52 @@ import ( "github.com/numaproj/numaflow/pkg/isb" ) -// tracker is used to store a key value pair for string and *isb.ReadMessage +// Tracker is used to store a key value pair for string and *isb.ReadMessage // as it can be accessed by concurrent goroutines, we keep all operations // under a mutex -type tracker struct { +type Tracker struct { lock sync.RWMutex m map[string]*isb.ReadMessage } -// NewTracker initializes a new instance of a tracker -func NewTracker() *tracker { - return &tracker{ +// NewTracker initializes a new instance of a Tracker +func NewTracker() *Tracker { + return &Tracker{ m: make(map[string]*isb.ReadMessage), lock: sync.RWMutex{}, } } -// addRequest add a new entry for a given message to the tracker. +// AddRequest add a new entry for a given message to the Tracker. // the key is chosen as the read offset of the message -func (t *tracker) addRequest(msg *isb.ReadMessage) { +func (t *Tracker) AddRequest(msg *isb.ReadMessage) { id := msg.ReadOffset.String() t.set(id, msg) } -// getRequest returns the message corresponding to a given id, along with a bool +// GetRequest returns the message corresponding to a given id, along with a bool // to indicate if it does not exist -func (t *tracker) getRequest(id string) (*isb.ReadMessage, bool) { +func (t *Tracker) GetRequest(id string) (*isb.ReadMessage, bool) { return t.get(id) } -// removeRequest will remove the entry for a given id -func (t *tracker) removeRequest(id string) { +// RemoveRequest will remove the entry for a given id +func (t *Tracker) RemoveRequest(id string) { t.delete(id) } // get is a helper function which fetches the message corresponding to a given id // it acquires a lock before accessing the map -func (t *tracker) get(key string) (*isb.ReadMessage, bool) { +func (t *Tracker) get(key string) (*isb.ReadMessage, bool) { t.lock.RLock() defer t.lock.RUnlock() item, ok := t.m[key] return item, ok } -// set is a helper function which add a key, value pair to the tracker map +// set is a helper function which add a key, value pair to the Tracker map // it acquires a lock before accessing the map -func (t *tracker) set(key string, msg *isb.ReadMessage) { +func (t *Tracker) set(key string, msg *isb.ReadMessage) { t.lock.Lock() defer t.lock.Unlock() t.m[key] = msg @@ -59,15 +59,15 @@ func (t *tracker) set(key string, msg *isb.ReadMessage) { // delete is a helper function which will remove the entry for a given id // it acquires a lock before accessing the map -func (t *tracker) delete(key string) { +func (t *Tracker) delete(key string) { t.lock.Lock() defer t.lock.Unlock() delete(t.m, key) } -// isEmpty is a helper function which checks if the tracker map is empty +// IsEmpty is a helper function which checks if the Tracker map is empty // return true if empty -func (t *tracker) isEmpty() bool { +func (t *Tracker) IsEmpty() bool { t.lock.RLock() defer t.lock.RUnlock() items := len(t.m) diff --git a/pkg/udf/rpc/tracker_test.go b/pkg/udf/rpc/tracker_test.go index 21704f4425..2eb2029702 100644 --- a/pkg/udf/rpc/tracker_test.go +++ b/pkg/udf/rpc/tracker_test.go @@ -13,10 +13,10 @@ func TestTracker_AddRequest(t *testing.T) { tr := NewTracker() readMessages := testutils.BuildTestReadMessages(3, time.Unix(1661169600, 0), nil) for _, msg := range readMessages { - tr.addRequest(&msg) + tr.AddRequest(&msg) } id := readMessages[0].ReadOffset.String() - m, ok := tr.getRequest(id) + m, ok := tr.GetRequest(id) assert.True(t, ok) assert.Equal(t, readMessages[0], *m) } @@ -25,13 +25,13 @@ func TestTracker_RemoveRequest(t *testing.T) { tr := NewTracker() readMessages := testutils.BuildTestReadMessages(3, time.Unix(1661169600, 0), nil) for _, msg := range readMessages { - tr.addRequest(&msg) + tr.AddRequest(&msg) } id := readMessages[0].ReadOffset.String() - m, ok := tr.getRequest(id) + m, ok := tr.GetRequest(id) assert.True(t, ok) assert.Equal(t, readMessages[0], *m) - tr.removeRequest(id) - _, ok = tr.getRequest(id) + tr.RemoveRequest(id) + _, ok = tr.GetRequest(id) assert.False(t, ok) } From 357a06fad831badc2dd84223f213bcfcacd011cc Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 18:15:54 +0530 Subject: [PATCH 02/15] Fix tests - pkg/source/transformer Signed-off-by: Sreekanth --- .../transformer/grpc_transformer_test.go | 470 +++--------------- 1 file changed, 59 insertions(+), 411 deletions(-) diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 86d25c147c..600369b255 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -20,7 +20,6 @@ import ( "context" "encoding/json" "errors" - "fmt" "github.com/numaproj/numaflow-go/pkg/sourcetransformer" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/test/bufconn" @@ -28,96 +27,49 @@ import ( "testing" "time" - "github.com/golang/mock/gomock" transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" - transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" - "github.com/stretchr/testify/assert" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" - "google.golang.org/protobuf/proto" - "google.golang.org/protobuf/types/known/timestamppb" - "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" sourcetransformerSdk "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/udf/rpc" + "github.com/stretchr/testify/assert" + "google.golang.org/grpc" ) -func NewMockGRPCBasedTransformer(mockClient *transformermock.MockSourceTransformClient) *GRPCBasedTransformer { - c, _ := sourcetransformerSdk.NewFromClient(mockClient) - return &GRPCBasedTransformer{"test-vertex", c} -} - -func TestGRPCBasedTransformer_WaitUntilReadyWithMockClient(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&transformpb.ReadyResponse{Ready: true}, nil) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - err := u.WaitUntilReady(ctx) - assert.NoError(t, err) -} - -type rpcMsg struct { - msg proto.Message -} - -func (r *rpcMsg) Matches(msg interface{}) bool { - m, ok := msg.(proto.Message) - if !ok { - return false +func TestGRPCBasedTransformer_WaitUntilReadyWithServer(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.Messages{} + }), } - return proto.Equal(m, r.msg) -} -func (r *rpcMsg) String() string { - return fmt.Sprintf("is %s", r.msg) + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) + err := u.WaitUntilReady(context.Background()) + assert.NoError(t, err) } -func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { +func TestGRPCBasedTransformer_BasicApplyWithServer(t *testing.T) { t.Run("test success", func(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(datum.Value(), datum.EventTime()).WithKeys(keys)) + }), + } - ctrl := gomock.NewController(t) - defer ctrl.Finish() + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169600, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ - Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - }, - }, - }, nil) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() + ctx := context.Background() - u := NewMockGRPCBasedTransformer(mockClient) got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ @@ -138,90 +90,29 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) - assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) + assert.Equal(t, []string{"test_success_key"}, got[0].WriteMessages[0].Keys) + assert.Equal(t, []byte(`forward_message`), got[0].WriteMessages[0].Payload) }) t.Run("test error", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.Messages{} + }), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, fmt.Errorf("mock error")) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) }) - }) - - t.Run("test error retryable: failed after 5 retries", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() + // Create a context and cancel it immediately. + // This cancelled context is passed to the ApplyTransform function to simulate failure + ctx, cancel := context.WithCancel(context.Background()) + cancel() - u := NewMockGRPCBasedTransformer(mockClient) _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ @@ -241,173 +132,18 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), }}, ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - }) - }) - t.Run("test error retryable: failed after 1 retry", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.InvalidArgument, "mock test err: non retryable").Err()) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ + expectedUDFErr := &rpc.ApplyUDFErr{ UserUDFErr: false, - Message: fmt.Sprintf("%s", err), + Message: "gRPC client.SourceTransformFn failed, Canceled: context canceled", InternalErr: rpc.InternalErr{ Flag: true, MainCarDown: false, }, - }) - }) - - t.Run("test error retryable: success after 1 retry", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169720, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ - Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - }, - }, - }, nil) - - ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169720, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_success_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) - assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) - }) - - t.Run("test error non retryable", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.InvalidArgument, "mock test err: non retryable").Err()) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - }) + var receivedErr *rpc.ApplyUDFErr + assert.ErrorAs(t, err, &receivedErr) + assert.Equal(t, expectedUDFErr, receivedErr) }) } @@ -448,114 +184,26 @@ func TestGRPCBasedTransformer_ApplyWithServer_ChangePayload(t *testing.T) { apply, err := u.ApplyTransform(context.TODO(), messages) assert.NoError(t, err) - var results = make([][]byte, len(readMessages)) - var resultKeys = make([][]string, len(readMessages)) - for idx, pair := range apply { - results[idx] = pair.WriteMessages[0].Payload - resultKeys[idx] = pair.WriteMessages[0].Header.Keys - } - - var expectedResults = make([][]byte, count) - var expectedKeys = make([][]string, count) - for idx, readMessage := range readMessages { + for _, pair := range apply { + resultPayload := pair.WriteMessages[0].Payload + resultKeys := pair.WriteMessages[0].Header.Keys var readMessagePayload testutils.PayloadForTest - _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) + _ = json.Unmarshal(pair.ReadMessage.Payload, &readMessagePayload) + var expectedKeys []string if readMessagePayload.Value%2 == 0 { - expectedKeys[idx] = []string{"even"} + expectedKeys = []string{"even"} } else { - expectedKeys[idx] = []string{"odd"} + expectedKeys = []string{"odd"} } + assert.Equal(t, expectedKeys, resultKeys) + doubledValue := testutils.PayloadForTest{ Key: readMessagePayload.Key, Value: readMessagePayload.Value * 2, } marshal, _ := json.Marshal(doubledValue) - expectedResults[idx] = marshal - } - - assert.ElementsMatch(t, expectedResults, results) - assert.Equal(t, expectedKeys, resultKeys) -} - -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { - multiplyBy2 := func(body []byte) interface{} { - var result testutils.PayloadForTest - _ = json.Unmarshal(body, &result) - result.Value = result.Value * 2 - return result - } - - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *transformpb.SourceTransformRequest, opts ...grpc.CallOption) (*transformpb.SourceTransformResponse, error) { - var originalValue testutils.PayloadForTest - _ = json.Unmarshal(datum.GetValue(), &originalValue) - doubledValue, _ := json.Marshal(multiplyBy2(datum.GetValue()).(testutils.PayloadForTest)) - var Results []*transformpb.SourceTransformResponse_Result - if originalValue.Value%2 == 0 { - Results = append(Results, &transformpb.SourceTransformResponse_Result{ - Keys: []string{"even"}, - Value: doubledValue, - }) - } else { - Results = append(Results, &transformpb.SourceTransformResponse_Result{ - Keys: []string{"odd"}, - Value: doubledValue, - }) - } - datumList := &transformpb.SourceTransformResponse{ - Results: Results, - } - return datumList, nil - }, - ).AnyTimes() - - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - - var count = int64(10) - readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) - - var results = make([][]byte, len(readMessages)) - var resultKeys = make([][]string, len(readMessages)) - messages := make([]*isb.ReadMessage, len(readMessages)) - for idx, readMessage := range readMessages { - messages[idx] = &readMessage - } - apply, err := u.ApplyTransform(ctx, messages) - assert.NoError(t, err) - for idx, pair := range apply { - results[idx] = pair.WriteMessages[0].Payload - resultKeys[idx] = pair.WriteMessages[0].Header.Keys - } - - var expectedResults = make([][]byte, count) - var expectedKeys = make([][]string, count) - for idx, readMessage := range readMessages { - var readMessagePayload testutils.PayloadForTest - _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) - if readMessagePayload.Value%2 == 0 { - expectedKeys[idx] = []string{"even"} - } else { - expectedKeys[idx] = []string{"odd"} - } - marshal, _ := json.Marshal(multiplyBy2(readMessage.Payload)) - expectedResults[idx] = marshal + assert.Equal(t, marshal, resultPayload) } - - assert.Equal(t, expectedResults, results) - assert.Equal(t, expectedKeys, resultKeys) } func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { @@ -609,7 +257,7 @@ func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientCon return conn } -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { +func TestGRPCBasedTransformer_Apply_ChangeEventTime(t *testing.T) { testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) svc := &sourcetransformer.Service{ Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { From 19f801d6e5f40bfab18736efe9ba0e11da939da1 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 18:35:34 +0530 Subject: [PATCH 03/15] Code generation Signed-off-by: Sreekanth --- pkg/sources/forward/data_forward_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index e4fb00a2e0..96cb6760e6 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -883,7 +883,7 @@ func (f mySourceForwardTest) ApplyTransform(ctx context.Context, messages []*isb } results[i] = isb.ReadWriteMessagePair{ ReadMessage: message, - WriteMessages: []*isb.WriteMessage{&isb.WriteMessage{ + WriteMessages: []*isb.WriteMessage{{ Message: writeMsg, }}, } From ad732ff7465dc2379419055555e741a72c4eb383 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 15:56:23 +0530 Subject: [PATCH 04/15] Refactoring, fix syntax errors Signed-off-by: Sreekanth --- go.sum | 4 +- pkg/sdkclient/sourcetransformer/client.go | 57 +++- .../sourcetransformer/client_test.go | 137 +++++++--- pkg/sdkclient/sourcetransformer/interface.go | 2 +- .../forward/applier/sourcetransformer.go | 8 +- pkg/sources/forward/data_forward.go | 67 +---- pkg/sources/forward/data_forward_test.go | 91 ++++--- pkg/sources/forward/shutdown_test.go | 12 +- pkg/sources/transformer/grpc_transformer.go | 170 ++++++------ .../transformer/grpc_transformer_test.go | 250 +++++++++++++----- pkg/udf/rpc/grpc_batch_map.go | 24 +- pkg/udf/rpc/tracker.go | 34 +-- pkg/udf/rpc/tracker_test.go | 12 +- 13 files changed, 532 insertions(+), 336 deletions(-) diff --git a/go.sum b/go.sum index 77784a6a8d..5ebfdd5ebe 100644 --- a/go.sum +++ b/go.sum @@ -485,8 +485,8 @@ github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDm github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= github.com/niemeyer/pretty v0.0.0-20200227124842-a10e7caefd8e/go.mod h1:zD1mROLANZcx1PVRCS0qkT7pwLkGfwJo4zjcN/Tysno= -github.com/numaproj/numaflow-go v0.8.0 h1:1Pp0AMLXkmUPlvFjKeY3a9X+OLU8oN1OQWxD9jLg8Uo= -github.com/numaproj/numaflow-go v0.8.0/go.mod h1:WoMt31+h3up202zTRI8c/qe42B8UbvwLe2mJH0MAlhI= +github.com/numaproj/numaflow-go v0.8.2-0.20240916154529-f205113bba8e h1:ApgoGlSvmSo/+/P9XF/XeY7afWE0jSxAFgkeFaM6IW4= +github.com/numaproj/numaflow-go v0.8.2-0.20240916154529-f205113bba8e/go.mod h1:s4F+Xh4BKjSEdL+49sxKgJq8MQO41aVjN4Try4gsKwg= github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= github.com/nxadm/tail v1.4.8 h1:nPr65rt6Y5JFSKQO7qToXr7pePgD6Gwiw05lkbyAQTE= github.com/nxadm/tail v1.4.8/go.mod h1:+ncqLTQzXmGhMZNUePPaPqPvBxHAIsmXswZKocGu+AU= diff --git a/pkg/sdkclient/sourcetransformer/client.go b/pkg/sdkclient/sourcetransformer/client.go index d9d47302c0..0912ef733d 100644 --- a/pkg/sdkclient/sourcetransformer/client.go +++ b/pkg/sdkclient/sourcetransformer/client.go @@ -18,6 +18,8 @@ package sourcetransformer import ( "context" + "errors" + "io" "google.golang.org/grpc" "google.golang.org/protobuf/types/known/emptypb" @@ -64,7 +66,7 @@ func NewFromClient(c transformpb.SourceTransformClient) (Client, error) { } // CloseConn closes the grpc client connection. -func (c *client) CloseConn(ctx context.Context) error { +func (c *client) CloseConn(_ context.Context) error { if c.conn == nil { return nil } @@ -81,11 +83,54 @@ func (c *client) IsReady(ctx context.Context, in *emptypb.Empty) (bool, error) { } // SourceTransformFn SourceTransformerFn applies a function to each request element. -func (c *client) SourceTransformFn(ctx context.Context, request *transformpb.SourceTransformRequest) (*transformpb.SourceTransformResponse, error) { - transformResponse, err := c.grpcClt.SourceTransformFn(ctx, request) - err = sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) +func (c *client) SourceTransformFn(ctx context.Context, request <-chan *transformpb.SourceTransformRequest) (<-chan *transformpb.SourceTransformResponse, <-chan error) { + errCh := make(chan error) + responseCh := make(chan *transformpb.SourceTransformResponse) + stream, err := c.grpcClt.SourceTransformFn(ctx) if err != nil { - return nil, err + go func() { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) + }() + return nil, errCh } - return transformResponse, nil + + go func() { + defer close(responseCh) + for { + resp, err := stream.Recv() + if err != nil { + if errors.Is(err, io.EOF) { + errCh = nil + return + } + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn", err) + return + } + responseCh <- resp + } + }() + + go func() { + for { + select { + case <-ctx.Done(): + return + case msg, ok := <-request: + if !ok { + err := stream.CloseSend() + if err != nil && errors.Is(err, io.EOF) { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn stream.CloseSend", err) + } + return + } + err := stream.Send(msg) + if err != nil { + errCh <- sdkerr.ToUDFErr("c.grpcClt.SourceTransformFn stream.Send", err) + return + } + } + } + }() + + return responseCh, errCh } diff --git a/pkg/sdkclient/sourcetransformer/client_test.go b/pkg/sdkclient/sourcetransformer/client_test.go index 27526312fd..b528b037bc 100644 --- a/pkg/sdkclient/sourcetransformer/client_test.go +++ b/pkg/sdkclient/sourcetransformer/client_test.go @@ -18,15 +18,22 @@ package sourcetransformer import ( "context" + "errors" "fmt" - "reflect" - "testing" - "github.com/golang/mock/gomock" transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" + "github.com/numaproj/numaflow-go/pkg/sourcetransformer" "github.com/stretchr/testify/assert" + "google.golang.org/grpc" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/test/bufconn" "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/timestamppb" + "net" + "reflect" + "testing" + "time" ) func TestClient_IsReady(t *testing.T) { @@ -54,44 +61,100 @@ func TestClient_IsReady(t *testing.T) { assert.EqualError(t, err, "mock connection refused") } -func TestClient_SourceTransformFn(t *testing.T) { - var ctx = context.Background() +func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { + lis := bufconn.Listen(100) + t.Cleanup(func() { + _ = lis.Close() + }) - ctrl := gomock.NewController(t) - defer ctrl.Finish() + server := grpc.NewServer() + t.Cleanup(func() { + server.Stop() + }) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).Return(&transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, nil) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).Return(&transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, fmt.Errorf("mock connection refused")) + register(server) - testClient, err := NewFromClient(mockClient) - assert.NoError(t, err) - reflect.DeepEqual(testClient, &client{ - grpcClt: mockClient, + errChan := make(chan error, 1) + go func() { + // t.Fatal should only be called from the goroutine running the test + if err := server.Serve(lis); err != nil { + errChan <- err + } + }() + + dialer := func(context.Context, string) (net.Conn, error) { + return lis.Dial() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + t.Cleanup(func() { + cancel() }) - result, err := testClient.SourceTransformFn(ctx, &transformpb.SourceTransformRequest{}) - assert.Equal(t, &transformpb.SourceTransformResponse{Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"temp-key"}, - Value: []byte("mock result"), - Tags: nil, - }, - }}, result) - assert.NoError(t, err) + conn, err := grpc.DialContext(ctx, "", grpc.WithContextDialer(dialer), grpc.WithTransportCredentials(insecure.NewCredentials())) + t.Cleanup(func() { + _ = conn.Close() + }) + if err != nil { + t.Fatalf("Creating new gRPC client connection: %v", err) + } + + var grpcServerErr error + select { + case grpcServerErr = <-errChan: + case <-time.After(500 * time.Millisecond): + grpcServerErr = errors.New("gRPC server didn't start in 500ms") + } + if err != nil { + t.Fatalf("Failed to start gRPC server: %v", grpcServerErr) + } + + return conn +} - _, err = testClient.SourceTransformFn(ctx, &transformpb.SourceTransformRequest{}) - assert.EqualError(t, err, "NonRetryable: mock connection refused") +func TestClient_SourceTransformFn(t *testing.T) { + var testTime = time.Date(2021, 8, 15, 14, 30, 45, 100, time.Local) + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + msg := datum.Value() + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(msg, testTime).WithKeys([]string{keys[0] + "_test"})) + }), + } + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + var ctx = context.Background() + client, _ := NewFromClient(transformClient) + + reqChan := make(chan *transformpb.SourceTransformRequest, 1) + go func() { + for i := 0; i < 5; i++ { + reqChan <- &transformpb.SourceTransformRequest{ + Keys: []string{fmt.Sprintf("client_key_%d", i)}, + Value: []byte("test"), + } + } + }() + + respChan, errChan := client.SourceTransformFn(ctx, reqChan) + var results [][]*transformpb.SourceTransformResponse_Result + for i := 0; i < 5; i++ { + var resp *transformpb.SourceTransformResponse + var err error + select { + case resp = <-respChan: + case err = <-errChan: + } + assert.NoError(t, err) + results = append(results, resp.GetResults()) + } + expected := [][]*transformpb.SourceTransformResponse_Result{ + {{Keys: []string{"client_key_0_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_1_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_2_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_3_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + {{Keys: []string{"client_key_4_test"}, Value: []byte("test"), EventTime: timestamppb.New(testTime)}}, + } + assert.ElementsMatch(t, expected, results) } diff --git a/pkg/sdkclient/sourcetransformer/interface.go b/pkg/sdkclient/sourcetransformer/interface.go index 4d8e3d8f71..6006e380c8 100644 --- a/pkg/sdkclient/sourcetransformer/interface.go +++ b/pkg/sdkclient/sourcetransformer/interface.go @@ -27,5 +27,5 @@ import ( type Client interface { CloseConn(ctx context.Context) error IsReady(ctx context.Context, in *emptypb.Empty) (bool, error) - SourceTransformFn(ctx context.Context, request *transformpb.SourceTransformRequest) (*transformpb.SourceTransformResponse, error) + SourceTransformFn(ctx context.Context, request <-chan *transformpb.SourceTransformRequest) (<-chan *transformpb.SourceTransformResponse, <-chan error) } diff --git a/pkg/sources/forward/applier/sourcetransformer.go b/pkg/sources/forward/applier/sourcetransformer.go index 795cd4c5a2..a935d511ea 100644 --- a/pkg/sources/forward/applier/sourcetransformer.go +++ b/pkg/sources/forward/applier/sourcetransformer.go @@ -25,13 +25,13 @@ import ( // SourceTransformApplier applies the source transform on the read message and gives back a new message. Any UserError will be retried here, while // InternalErr can be returned and could be retried by the callee. type SourceTransformApplier interface { - ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) + ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) } // ApplySourceTransformFunc is a function type that implements SourceTransformApplier interface. -type ApplySourceTransformFunc func(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) +type ApplySourceTransformFunc func(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) // ApplyTransform implements SourceTransformApplier interface. -func (f ApplySourceTransformFunc) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return f(ctx, message) +func (f ApplySourceTransformFunc) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + return f(ctx, messages) } diff --git a/pkg/sources/forward/data_forward.go b/pkg/sources/forward/data_forward.go index 48ff97da3a..9eddafa30d 100644 --- a/pkg/sources/forward/data_forward.go +++ b/pkg/sources/forward/data_forward.go @@ -305,34 +305,14 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // If a user-defined transformer exists, apply it if df.opts.transformer != nil { - // user-defined transformer concurrent processing request channel - transformerCh := make(chan *isb.ReadWriteMessagePair) - - // create a pool of Transformer Processors - var wg sync.WaitGroup - for i := 0; i < df.opts.transformerConcurrency; i++ { - wg.Add(1) - go func() { - defer wg.Done() - df.concurrentApplyTransformer(ctx, transformerCh) - }() + for _, m := range readMessages { + // assign watermark to the message + m.Watermark = time.Time(processorWM) } concurrentTransformerProcessingStart := time.Now() - for idx, m := range readMessages { + readWriteMessagePairs = df.applyTransformer(ctx, readMessages) - // assign watermark to the message - m.Watermark = time.Time(processorWM) - readWriteMessagePairs[idx].ReadMessage = m - // send transformer processing work to the channel. Thus, the results of the transformer - // application on a read message will be stored as the corresponding writeMessage in readWriteMessagePairs - transformerCh <- &readWriteMessagePairs[idx] - } - // let the go routines know that there is no more work - close(transformerCh) - // wait till the processing is done. this will not be an infinite wait because the transformer processing will exit if - // context.Done() is closed. - wg.Wait() df.opts.logger.Debugw("concurrent applyTransformer completed", zap.Int("concurrency", df.opts.transformerConcurrency), zap.Duration("took", time.Since(concurrentTransformerProcessingStart)), @@ -661,42 +641,12 @@ func (df *DataForward) writeToBuffer(ctx context.Context, toBufferPartition isb. return writeOffsets, nil } -// concurrentApplyTransformer applies the transformer based on the request from the channel -func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessagePair <-chan *isb.ReadWriteMessagePair) { - for message := range readMessagePair { - start := time.Now() - metrics.SourceTransformerReadMessagesCount.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Inc() - - writeMessages, err := df.applyTransformer(ctx, message.ReadMessage) - metrics.SourceTransformerWriteMessagesCount.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Add(float64(len(writeMessages))) - - message.WriteMessages = append(message.WriteMessages, writeMessages...) - message.Err = err - metrics.SourceTransformerProcessingTime.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - metrics.LabelPartitionName: df.reader.GetName(), - }).Observe(float64(time.Since(start).Microseconds())) - } -} - // applyTransformer applies the transformer and will block if there is any InternalErr. On the other hand, if this is a UserError // the skip flag is set. The ShutDown flag will only if there is an InternalErr and ForceStop has been invoked. // The UserError retry will be done on the applyTransformer. -func (df *DataForward) applyTransformer(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (df *DataForward) applyTransformer(ctx context.Context, messages []*isb.ReadMessage) []isb.ReadWriteMessagePair { for { - writeMessages, err := df.opts.transformer.ApplyTransform(ctx, readMessage) + transformResults, err := df.opts.transformer.ApplyTransform(ctx, messages) if err != nil { df.opts.logger.Errorw("Transformer.Apply error", zap.Error(err)) // TODO: implement retry with backoff etc. @@ -712,12 +662,11 @@ func (df *DataForward) applyTransformer(ctx context.Context, readMessage *isb.Re metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), }).Inc() - - return nil, err + return []isb.ReadWriteMessagePair{{Err: err}} } continue } - return writeMessages, nil + return transformResults } } diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index 25e41a9fa6..e4fb00a2e0 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -121,8 +121,16 @@ func (f myForwardTest) WhereTo(_ []string, _ []string, s string) ([]forwarder.Ve }}, nil } -func (f myForwardTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + out := make([]isb.ReadWriteMessagePair, len(messages)) + for i, msg := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", msg) + out[i] = isb.ReadWriteMessagePair{ + ReadMessage: msg, + WriteMessages: writeMsg, + } + } + return out, nil } func TestNewDataForward(t *testing.T) { @@ -856,36 +864,31 @@ func (f *mySourceForwardTestRoundRobin) WhereTo(_ []string, _ []string, s string // such that we can verify message IsLate attribute gets set to true. var testSourceNewEventTime = testSourceWatermark.Add(time.Duration(-1) * time.Minute) -func (f mySourceForwardTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return func(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { - _ = ctx - offset := readMessage.ReadOffset - payload := readMessage.Body.Payload - parentPaneInfo := readMessage.MessageInfo - - // apply source data transformer - _ = payload - // copy the payload - result := payload - // assign new event time - parentPaneInfo.EventTime = testSourceNewEventTime - var key []string - - writeMessage := isb.Message{ +func (f mySourceForwardTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + message.MessageInfo.EventTime = testSourceNewEventTime + writeMsg := isb.Message{ Header: isb.Header{ - MessageInfo: parentPaneInfo, + MessageInfo: message.MessageInfo, ID: isb.MessageID{ VertexName: "test-vertex", - Offset: offset.String(), + Offset: message.ReadOffset.String(), }, - Keys: key, + Keys: []string{}, }, Body: isb.Body{ - Payload: result, + Payload: message.Body.Payload, }, } - return []*isb.WriteMessage{{Message: writeMessage}}, nil - }(ctx, message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: []*isb.WriteMessage{&isb.WriteMessage{ + Message: writeMsg, + }}, + } + } + return results, nil } // TestSourceWatermarkPublisher is a dummy implementation of isb.SourceWatermarkPublisher interface @@ -1153,8 +1156,16 @@ func (f myForwardDropTest) WhereTo(_ []string, _ []string, s string) ([]forwarde return []forwarder.VertexBuffer{}, nil } -func (f myForwardDropTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardDropTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardToAllTest struct { @@ -1174,8 +1185,16 @@ func (f *myForwardToAllTest) WhereTo(_ []string, _ []string, s string) ([]forwar return output, nil } -func (f *myForwardToAllTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f *myForwardToAllTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardInternalErrTest struct { @@ -1188,7 +1207,7 @@ func (f myForwardInternalErrTest) WhereTo(_ []string, _ []string, s string) ([]f }}, nil } -func (f myForwardInternalErrTest) ApplyTransform(_ context.Context, _ *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (f myForwardInternalErrTest) ApplyTransform(ctx context.Context, _ []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { return nil, &udfapplier.ApplyUDFErr{ UserUDFErr: false, InternalErr: struct { @@ -1209,8 +1228,16 @@ func (f myForwardApplyWhereToErrTest) WhereTo(_ []string, _ []string, s string) }}, fmt.Errorf("whereToStep failed") } -func (f myForwardApplyWhereToErrTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "test-vertex", message) +func (f myForwardApplyWhereToErrTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "test-vertex", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } type myForwardApplyTransformerErrTest struct { @@ -1223,7 +1250,7 @@ func (f myForwardApplyTransformerErrTest) WhereTo(_ []string, _ []string, s stri }}, nil } -func (f myForwardApplyTransformerErrTest) ApplyTransform(_ context.Context, _ *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func (f myForwardApplyTransformerErrTest) ApplyTransform(_ context.Context, _ []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { return nil, fmt.Errorf("transformer error") } diff --git a/pkg/sources/forward/shutdown_test.go b/pkg/sources/forward/shutdown_test.go index a4ffc5e2e2..34003e729f 100644 --- a/pkg/sources/forward/shutdown_test.go +++ b/pkg/sources/forward/shutdown_test.go @@ -43,8 +43,16 @@ func (s myShutdownTest) WhereTo([]string, []string, string) ([]forwarder.VertexB return []forwarder.VertexBuffer{}, nil } -func (s myShutdownTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, "", message) +func (f myShutdownTest) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + results := make([]isb.ReadWriteMessagePair, len(messages)) + for i, message := range messages { + writeMsg, _ := testutils.CopyUDFTestApply(ctx, "", message) + results[i] = isb.ReadWriteMessagePair{ + ReadMessage: message, + WriteMessages: writeMsg, + } + } + return results, nil } func (s myShutdownTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { diff --git a/pkg/sources/transformer/grpc_transformer.go b/pkg/sources/transformer/grpc_transformer.go index 14b414a348..c7f9902603 100644 --- a/pkg/sources/transformer/grpc_transformer.go +++ b/pkg/sources/transformer/grpc_transformer.go @@ -18,19 +18,17 @@ package transformer import ( "context" + "errors" "fmt" "time" v1 "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" - "google.golang.org/protobuf/types/known/emptypb" - "google.golang.org/protobuf/types/known/timestamppb" - "k8s.io/apimachinery/pkg/util/wait" - "github.com/numaproj/numaflow/pkg/isb" - sdkerr "github.com/numaproj/numaflow/pkg/sdkclient/error" "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/rpc" + "google.golang.org/protobuf/types/known/emptypb" + "google.golang.org/protobuf/types/known/timestamppb" ) // GRPCBasedTransformer applies user-defined transformer over gRPC (over Unix Domain Socket) client/server where server is the transformer. @@ -75,59 +73,36 @@ func (u *GRPCBasedTransformer) CloseConn(ctx context.Context) error { return u.client.CloseConn(ctx) } -func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { - keys := readMessage.Keys - payload := readMessage.Body.Payload - offset := readMessage.ReadOffset - parentMessageInfo := readMessage.MessageInfo - var req = &v1.SourceTransformRequest{ - Keys: keys, - Value: payload, - EventTime: timestamppb.New(parentMessageInfo.EventTime), - Watermark: timestamppb.New(readMessage.Watermark), - Headers: readMessage.Headers, - } +func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, messages []*isb.ReadMessage) ([]isb.ReadWriteMessagePair, error) { + var transformResults []isb.ReadWriteMessagePair + inputChan := make(chan *v1.SourceTransformRequest) + respChan, errChan := u.client.SourceTransformFn(ctx, inputChan) - response, err := u.client.SourceTransformFn(ctx, req) - if err != nil { - udfErr, _ := sdkerr.FromError(err) - switch udfErr.ErrorKind() { - case sdkerr.Retryable: - var success bool - _ = wait.ExponentialBackoffWithContext(ctx, wait.Backoff{ - // retry every "duration * factor + [0, jitter]" interval for 5 times - Duration: 1 * time.Second, - Factor: 1, - Jitter: 0.1, - Steps: 5, - }, func(_ context.Context) (done bool, err error) { - response, err = u.client.SourceTransformFn(ctx, req) - if err != nil { - udfErr, _ = sdkerr.FromError(err) - switch udfErr.ErrorKind() { - case sdkerr.Retryable: - return false, nil - case sdkerr.NonRetryable: - return true, nil - default: - return true, nil - } - } - success = true - return true, nil - }) - if !success { - return nil, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - } + tracker := rpc.NewTracker() + + go func() { + defer close(inputChan) + for _, msg := range messages { + tracker.AddRequest(msg) + req := &v1.SourceTransformRequest{ + Keys: msg.Keys, + Value: msg.Body.Payload, + EventTime: timestamppb.New(msg.MessageInfo.EventTime), + Watermark: timestamppb.New(msg.Watermark), + Headers: msg.Headers, + Id: msg.ReadOffset.String(), } - case sdkerr.NonRetryable: - return nil, &rpc.ApplyUDFErr{ + inputChan <- req + } + }() + + messageCount := len(messages) + +loop: + for { + select { + case err := <-errChan: + err = &rpc.ApplyUDFErr{ UserUDFErr: false, Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), InternalErr: rpc.InternalErr{ @@ -135,43 +110,58 @@ func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, readMessage * MainCarDown: false, }, } - default: - return nil, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("gRPC client.SourceTransformFn failed, %s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, + return nil, err + case resp, ok := <-respChan: + if !ok { + break loop + } + msgId := resp.GetId() + parentMessage, ok := tracker.GetRequest(msgId) + if !ok { + return nil, errors.New("tracker doesn't contain the message ID received from the response") + } + var taggedMessages []*isb.WriteMessage + for i, result := range resp.GetResults() { + keys := result.Keys + if result.EventTime != nil { + // Transformer supports changing event time. + parentMessage.MessageInfo.EventTime = result.EventTime.AsTime() + } + taggedMessage := &isb.WriteMessage{ + Message: isb.Message{ + Header: isb.Header{ + MessageInfo: parentMessage.MessageInfo, + ID: isb.MessageID{ + VertexName: u.vertexName, + Offset: parentMessage.ReadOffset.String(), + Index: int32(i), + }, + Keys: keys, + }, + Body: isb.Body{ + Payload: result.Value, + }, + }, + Tags: result.Tags, + } + taggedMessages = append(taggedMessages, taggedMessage) + } + responsePair := isb.ReadWriteMessagePair{ + ReadMessage: parentMessage, + WriteMessages: taggedMessages, + Err: nil, + } + transformResults = append(transformResults, responsePair) + tracker.RemoveRequest(msgId) + messageCount-- + if messageCount == 0 { + break loop } } } - taggedMessages := make([]*isb.WriteMessage, 0) - for i, result := range response.GetResults() { - keys := result.Keys - if result.EventTime != nil { - // Transformer supports changing event time. - parentMessageInfo.EventTime = result.EventTime.AsTime() - } - taggedMessage := &isb.WriteMessage{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: parentMessageInfo, - ID: isb.MessageID{ - VertexName: u.vertexName, - Offset: offset.String(), - Index: int32(i), - }, - Keys: keys, - }, - Body: isb.Body{ - Payload: result.Value, - }, - }, - Tags: result.Tags, - } - taggedMessages = append(taggedMessages, taggedMessage) + if !tracker.IsEmpty() { + return nil, errors.New("transform response for all requests were not received from UDF") } - return taggedMessages, nil + return transformResults, nil } diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 959a40bf51..86d25c147c 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -19,12 +19,17 @@ package transformer import ( "context" "encoding/json" + "errors" "fmt" + "github.com/numaproj/numaflow-go/pkg/sourcetransformer" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/test/bufconn" + "net" "testing" "time" "github.com/golang/mock/gomock" - v1 "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" + transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" "github.com/stretchr/testify/assert" "google.golang.org/grpc" @@ -35,12 +40,12 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" - "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" + sourcetransformerSdk "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/udf/rpc" ) func NewMockGRPCBasedTransformer(mockClient *transformermock.MockSourceTransformClient) *GRPCBasedTransformer { - c, _ := sourcetransformer.NewFromClient(mockClient) + c, _ := sourcetransformerSdk.NewFromClient(mockClient) return &GRPCBasedTransformer{"test-vertex", c} } @@ -49,7 +54,7 @@ func TestGRPCBasedTransformer_WaitUntilReadyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&v1.ReadyResponse{Ready: true}, nil) + mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&transformpb.ReadyResponse{Ready: true}, nil) ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() @@ -88,14 +93,14 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169600, 0)), Watermark: timestamppb.New(time.Time{}), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&v1.SourceTransformResponse{ - Results: []*v1.SourceTransformResponse_Result{ + mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ + Results: []*transformpb.SourceTransformResponse_Result{ { Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), @@ -113,7 +118,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -130,11 +135,11 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].Keys) - assert.Equal(t, req.Value, got[0].Payload) + assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) + assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) }) t.Run("test error", func(t *testing.T) { @@ -142,7 +147,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -160,7 +165,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -177,7 +182,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -194,7 +199,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -217,7 +222,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -234,7 +239,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -251,7 +256,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -270,7 +275,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -287,7 +292,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -304,15 +309,15 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169720, 0)), Watermark: timestamppb.New(time.Time{}), } mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&v1.SourceTransformResponse{ - Results: []*v1.SourceTransformResponse_Result{ + mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ + Results: []*transformpb.SourceTransformResponse_Result{ { Keys: []string{"test_success_key"}, Value: []byte(`forward_message`), @@ -330,7 +335,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -347,11 +352,11 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].Keys) - assert.Equal(t, req.Value, got[0].Payload) + assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) + assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) }) t.Run("test error non retryable", func(t *testing.T) { @@ -359,7 +364,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { defer ctrl.Finish() mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &v1.SourceTransformRequest{ + req := &transformpb.SourceTransformRequest{ Keys: []string{"test_error_key"}, Value: []byte(`forward_message`), EventTime: timestamppb.New(time.Unix(1661169660, 0)), @@ -376,7 +381,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }() u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, &isb.ReadMessage{ + _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{ @@ -393,7 +398,7 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }, }, ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }, + }}, ) assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ UserUDFErr: false, @@ -406,6 +411,72 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }) } +func TestGRPCBasedTransformer_ApplyWithServer_ChangePayload(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + var originalValue testutils.PayloadForTest + _ = json.Unmarshal(datum.Value(), &originalValue) + doubledValue := testutils.PayloadForTest{ + Value: originalValue.Value * 2, + Key: originalValue.Key, + } + doubledValueBytes, _ := json.Marshal(&doubledValue) + + var resultKeys []string + if originalValue.Value%2 == 0 { + resultKeys = []string{"even"} + } else { + resultKeys = []string{"odd"} + } + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(doubledValueBytes, datum.EventTime()).WithKeys(resultKeys)) + }), + } + + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) + + var count = int64(10) + readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) + messages := make([]*isb.ReadMessage, len(readMessages)) + for idx, readMessage := range readMessages { + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(context.TODO(), messages) + assert.NoError(t, err) + + var results = make([][]byte, len(readMessages)) + var resultKeys = make([][]string, len(readMessages)) + for idx, pair := range apply { + results[idx] = pair.WriteMessages[0].Payload + resultKeys[idx] = pair.WriteMessages[0].Header.Keys + } + + var expectedResults = make([][]byte, count) + var expectedKeys = make([][]string, count) + for idx, readMessage := range readMessages { + var readMessagePayload testutils.PayloadForTest + _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) + if readMessagePayload.Value%2 == 0 { + expectedKeys[idx] = []string{"even"} + } else { + expectedKeys[idx] = []string{"odd"} + } + doubledValue := testutils.PayloadForTest{ + Key: readMessagePayload.Key, + Value: readMessagePayload.Value * 2, + } + marshal, _ := json.Marshal(doubledValue) + expectedResults[idx] = marshal + } + + assert.ElementsMatch(t, expectedResults, results) + assert.Equal(t, expectedKeys, resultKeys) +} + func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { multiplyBy2 := func(body []byte) interface{} { var result testutils.PayloadForTest @@ -419,23 +490,23 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { mockClient := transformermock.NewMockSourceTransformClient(ctrl) mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *v1.SourceTransformRequest, opts ...grpc.CallOption) (*v1.SourceTransformResponse, error) { + func(_ context.Context, datum *transformpb.SourceTransformRequest, opts ...grpc.CallOption) (*transformpb.SourceTransformResponse, error) { var originalValue testutils.PayloadForTest _ = json.Unmarshal(datum.GetValue(), &originalValue) doubledValue, _ := json.Marshal(multiplyBy2(datum.GetValue()).(testutils.PayloadForTest)) - var Results []*v1.SourceTransformResponse_Result + var Results []*transformpb.SourceTransformResponse_Result if originalValue.Value%2 == 0 { - Results = append(Results, &v1.SourceTransformResponse_Result{ + Results = append(Results, &transformpb.SourceTransformResponse_Result{ Keys: []string{"even"}, Value: doubledValue, }) } else { - Results = append(Results, &v1.SourceTransformResponse_Result{ + Results = append(Results, &transformpb.SourceTransformResponse_Result{ Keys: []string{"odd"}, Value: doubledValue, }) } - datumList := &v1.SourceTransformResponse{ + datumList := &transformpb.SourceTransformResponse{ Results: Results, } return datumList, nil @@ -458,11 +529,15 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { var results = make([][]byte, len(readMessages)) var resultKeys = make([][]string, len(readMessages)) + messages := make([]*isb.ReadMessage, len(readMessages)) for idx, readMessage := range readMessages { - apply, err := u.ApplyTransform(ctx, &readMessage) - assert.NoError(t, err) - results[idx] = apply[0].Payload - resultKeys[idx] = apply[0].Header.Keys + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(ctx, messages) + assert.NoError(t, err) + for idx, pair := range apply { + results[idx] = pair.WriteMessages[0].Payload + resultKeys[idx] = pair.WriteMessages[0].Header.Keys } var expectedResults = make([][]byte, count) @@ -483,43 +558,82 @@ func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { assert.Equal(t, expectedKeys, resultKeys) } -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { - testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) - ctrl := gomock.NewController(t) - defer ctrl.Finish() +func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { + lis := bufconn.Listen(100) + t.Cleanup(func() { + _ = lis.Close() + }) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *v1.SourceTransformRequest, opts ...grpc.CallOption) (*v1.SourceTransformResponse, error) { - var Results []*v1.SourceTransformResponse_Result - Results = append(Results, &v1.SourceTransformResponse_Result{ - Keys: []string{"even"}, - Value: datum.Value, - EventTime: timestamppb.New(testEventTime), - }) - datumList := &v1.SourceTransformResponse{ - Results: Results, - } - return datumList, nil - }, - ).AnyTimes() + server := grpc.NewServer() + t.Cleanup(func() { + server.Stop() + }) - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() + register(server) + + errChan := make(chan error, 1) go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") + // t.Fatal should only be called from the goroutine running the test + if err := server.Serve(lis); err != nil { + errChan <- err } }() - u := NewMockGRPCBasedTransformer(mockClient) + dialer := func(context.Context, string) (net.Conn, error) { + return lis.Dial() + } + + ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second) + t.Cleanup(func() { + cancel() + }) + + conn, err := grpc.DialContext(ctx, "", grpc.WithContextDialer(dialer), grpc.WithTransportCredentials(insecure.NewCredentials())) + t.Cleanup(func() { + _ = conn.Close() + }) + if err != nil { + t.Fatalf("Creating new gRPC client connection: %v", err) + } + + var grpcServerErr error + select { + case grpcServerErr = <-errChan: + case <-time.After(500 * time.Millisecond): + grpcServerErr = errors.New("gRPC server didn't start in 500ms") + } + if err != nil { + t.Fatalf("Failed to start gRPC server: %v", grpcServerErr) + } + + return conn +} + +func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { + testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + msg := datum.Value() + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(msg, testEventTime).WithKeys([]string{"even"})) + }), + } + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) var count = int64(2) readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) - for _, readMessage := range readMessages { - apply, err := u.ApplyTransform(ctx, &readMessage) - assert.NoError(t, err) - assert.Equal(t, testEventTime, apply[0].EventTime) + messages := make([]*isb.ReadMessage, len(readMessages)) + for idx, readMessage := range readMessages { + messages[idx] = &readMessage + } + apply, err := u.ApplyTransform(context.TODO(), messages) + assert.NoError(t, err) + for _, pair := range apply { + assert.NoError(t, pair.Err) + assert.Equal(t, testEventTime, pair.WriteMessages[0].EventTime) } } diff --git a/pkg/udf/rpc/grpc_batch_map.go b/pkg/udf/rpc/grpc_batch_map.go index 6d6c397642..8d072a73c8 100644 --- a/pkg/udf/rpc/grpc_batch_map.go +++ b/pkg/udf/rpc/grpc_batch_map.go @@ -34,7 +34,7 @@ import ( type GRPCBasedBatchMap struct { vertexName string client batchmapper.Client - requestTracker *tracker + requestTracker *Tracker } func NewUDSgRPCBasedBatchMap(vertexName string, client batchmapper.Client) *GRPCBasedBatchMap { @@ -44,7 +44,7 @@ func NewUDSgRPCBasedBatchMap(vertexName string, client batchmapper.Client) *GRPC // requestTracker is used to store the read messages in a key, value manner where // key is the read offset and the reference to read message as the value. // Once the results are received from the UDF, we map the responses to the corresponding request - // using a lookup on this tracker. + // using a lookup on this Tracker. requestTracker: NewTracker(), } } @@ -93,18 +93,18 @@ func (u *GRPCBasedBatchMap) ApplyBatchMap(ctx context.Context, messages []*isb.R // trackerReq is used to store the read messages in a key, value manner where // key is the read offset and the reference to read message as the value. // Once the results are received from the UDF, we map the responses to the corresponding request - // using a lookup on this tracker. + // using a lookup on this Tracker. trackerReq := NewTracker() // Read routine: this goroutine iterates over the input messages and sends each // of the read messages to the grpc client after transforming it to a BatchMapRequest. // Once all messages are sent, it closes the input channel to indicate that all requests have been read. - // On creating a new request, we add it to a tracker map so that the responses on the stream + // On creating a new request, we add it to a Tracker map so that the responses on the stream // can be mapped backed to the given parent request go func() { defer close(inputChan) for _, msg := range messages { - trackerReq.addRequest(msg) + trackerReq.AddRequest(msg) inputChan <- u.parseInputRequest(msg) } }() @@ -139,14 +139,14 @@ loop: } // Get the unique request ID for which these responses are meant for. msgId := grpcResp.GetId() - // Fetch the request value for the given ID from the tracker - parentMessage, ok := trackerReq.getRequest(msgId) + // Fetch the request value for the given ID from the Tracker + parentMessage, ok := trackerReq.GetRequest(msgId) if !ok { - // this case is when the given request ID was not present in the tracker. + // this case is when the given request ID was not present in the Tracker. // This means that either the UDF added an incorrect ID // This cannot be processed further and should result in an error // Can there be another case for this? - logger.Error("Request missing from tracker, ", msgId) + logger.Error("Request missing from Tracker, ", msgId) return nil, fmt.Errorf("incorrect ID found during batch map processing") } // parse the responses received @@ -159,12 +159,12 @@ loop: Err: nil, } udfResults = append(udfResults, responsePair) - trackerReq.removeRequest(msgId) + trackerReq.RemoveRequest(msgId) } } - // check if there are elements left in the tracker. This cannot be an acceptable case as we want the + // check if there are elements left in the Tracker. This cannot be an acceptable case as we want the // UDF to send responses for all elements. - if !trackerReq.isEmpty() { + if !trackerReq.IsEmpty() { logger.Error("BatchMap response for all requests not received from UDF") return nil, fmt.Errorf("batchMap response for all requests not received from UDF") } diff --git a/pkg/udf/rpc/tracker.go b/pkg/udf/rpc/tracker.go index 60b57a7af9..0065f02eb1 100644 --- a/pkg/udf/rpc/tracker.go +++ b/pkg/udf/rpc/tracker.go @@ -6,52 +6,52 @@ import ( "github.com/numaproj/numaflow/pkg/isb" ) -// tracker is used to store a key value pair for string and *isb.ReadMessage +// Tracker is used to store a key value pair for string and *isb.ReadMessage // as it can be accessed by concurrent goroutines, we keep all operations // under a mutex -type tracker struct { +type Tracker struct { lock sync.RWMutex m map[string]*isb.ReadMessage } -// NewTracker initializes a new instance of a tracker -func NewTracker() *tracker { - return &tracker{ +// NewTracker initializes a new instance of a Tracker +func NewTracker() *Tracker { + return &Tracker{ m: make(map[string]*isb.ReadMessage), lock: sync.RWMutex{}, } } -// addRequest add a new entry for a given message to the tracker. +// AddRequest add a new entry for a given message to the Tracker. // the key is chosen as the read offset of the message -func (t *tracker) addRequest(msg *isb.ReadMessage) { +func (t *Tracker) AddRequest(msg *isb.ReadMessage) { id := msg.ReadOffset.String() t.set(id, msg) } -// getRequest returns the message corresponding to a given id, along with a bool +// GetRequest returns the message corresponding to a given id, along with a bool // to indicate if it does not exist -func (t *tracker) getRequest(id string) (*isb.ReadMessage, bool) { +func (t *Tracker) GetRequest(id string) (*isb.ReadMessage, bool) { return t.get(id) } -// removeRequest will remove the entry for a given id -func (t *tracker) removeRequest(id string) { +// RemoveRequest will remove the entry for a given id +func (t *Tracker) RemoveRequest(id string) { t.delete(id) } // get is a helper function which fetches the message corresponding to a given id // it acquires a lock before accessing the map -func (t *tracker) get(key string) (*isb.ReadMessage, bool) { +func (t *Tracker) get(key string) (*isb.ReadMessage, bool) { t.lock.RLock() defer t.lock.RUnlock() item, ok := t.m[key] return item, ok } -// set is a helper function which add a key, value pair to the tracker map +// set is a helper function which add a key, value pair to the Tracker map // it acquires a lock before accessing the map -func (t *tracker) set(key string, msg *isb.ReadMessage) { +func (t *Tracker) set(key string, msg *isb.ReadMessage) { t.lock.Lock() defer t.lock.Unlock() t.m[key] = msg @@ -59,15 +59,15 @@ func (t *tracker) set(key string, msg *isb.ReadMessage) { // delete is a helper function which will remove the entry for a given id // it acquires a lock before accessing the map -func (t *tracker) delete(key string) { +func (t *Tracker) delete(key string) { t.lock.Lock() defer t.lock.Unlock() delete(t.m, key) } -// isEmpty is a helper function which checks if the tracker map is empty +// IsEmpty is a helper function which checks if the Tracker map is empty // return true if empty -func (t *tracker) isEmpty() bool { +func (t *Tracker) IsEmpty() bool { t.lock.RLock() defer t.lock.RUnlock() items := len(t.m) diff --git a/pkg/udf/rpc/tracker_test.go b/pkg/udf/rpc/tracker_test.go index 21704f4425..2eb2029702 100644 --- a/pkg/udf/rpc/tracker_test.go +++ b/pkg/udf/rpc/tracker_test.go @@ -13,10 +13,10 @@ func TestTracker_AddRequest(t *testing.T) { tr := NewTracker() readMessages := testutils.BuildTestReadMessages(3, time.Unix(1661169600, 0), nil) for _, msg := range readMessages { - tr.addRequest(&msg) + tr.AddRequest(&msg) } id := readMessages[0].ReadOffset.String() - m, ok := tr.getRequest(id) + m, ok := tr.GetRequest(id) assert.True(t, ok) assert.Equal(t, readMessages[0], *m) } @@ -25,13 +25,13 @@ func TestTracker_RemoveRequest(t *testing.T) { tr := NewTracker() readMessages := testutils.BuildTestReadMessages(3, time.Unix(1661169600, 0), nil) for _, msg := range readMessages { - tr.addRequest(&msg) + tr.AddRequest(&msg) } id := readMessages[0].ReadOffset.String() - m, ok := tr.getRequest(id) + m, ok := tr.GetRequest(id) assert.True(t, ok) assert.Equal(t, readMessages[0], *m) - tr.removeRequest(id) - _, ok = tr.getRequest(id) + tr.RemoveRequest(id) + _, ok = tr.GetRequest(id) assert.False(t, ok) } From a4a541b13caa468e69221eebdf8042b76e994475 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 18:15:54 +0530 Subject: [PATCH 05/15] Fix tests - pkg/source/transformer Signed-off-by: Sreekanth --- .../transformer/grpc_transformer_test.go | 470 +++--------------- 1 file changed, 59 insertions(+), 411 deletions(-) diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 86d25c147c..600369b255 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -20,7 +20,6 @@ import ( "context" "encoding/json" "errors" - "fmt" "github.com/numaproj/numaflow-go/pkg/sourcetransformer" "google.golang.org/grpc/credentials/insecure" "google.golang.org/grpc/test/bufconn" @@ -28,96 +27,49 @@ import ( "testing" "time" - "github.com/golang/mock/gomock" transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" - transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" - "github.com/stretchr/testify/assert" - "google.golang.org/grpc" - "google.golang.org/grpc/codes" - "google.golang.org/grpc/status" - "google.golang.org/protobuf/proto" - "google.golang.org/protobuf/types/known/timestamppb" - "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" sourcetransformerSdk "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" "github.com/numaproj/numaflow/pkg/udf/rpc" + "github.com/stretchr/testify/assert" + "google.golang.org/grpc" ) -func NewMockGRPCBasedTransformer(mockClient *transformermock.MockSourceTransformClient) *GRPCBasedTransformer { - c, _ := sourcetransformerSdk.NewFromClient(mockClient) - return &GRPCBasedTransformer{"test-vertex", c} -} - -func TestGRPCBasedTransformer_WaitUntilReadyWithMockClient(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().IsReady(gomock.Any(), gomock.Any()).Return(&transformpb.ReadyResponse{Ready: true}, nil) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - err := u.WaitUntilReady(ctx) - assert.NoError(t, err) -} - -type rpcMsg struct { - msg proto.Message -} - -func (r *rpcMsg) Matches(msg interface{}) bool { - m, ok := msg.(proto.Message) - if !ok { - return false +func TestGRPCBasedTransformer_WaitUntilReadyWithServer(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.Messages{} + }), } - return proto.Equal(m, r.msg) -} -func (r *rpcMsg) String() string { - return fmt.Sprintf("is %s", r.msg) + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) + err := u.WaitUntilReady(context.Background()) + assert.NoError(t, err) } -func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { +func TestGRPCBasedTransformer_BasicApplyWithServer(t *testing.T) { t.Run("test success", func(t *testing.T) { + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.MessagesBuilder().Append(sourcetransformer.NewMessage(datum.Value(), datum.EventTime()).WithKeys(keys)) + }), + } - ctrl := gomock.NewController(t) - defer ctrl.Finish() + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) + }) + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169600, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ - Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - }, - }, - }, nil) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() + ctx := context.Background() - u := NewMockGRPCBasedTransformer(mockClient) got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ @@ -138,90 +90,29 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { }}, ) assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) - assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) + assert.Equal(t, []string{"test_success_key"}, got[0].WriteMessages[0].Keys) + assert.Equal(t, []byte(`forward_message`), got[0].WriteMessages[0].Payload) }) t.Run("test error", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), + svc := &sourcetransformer.Service{ + Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { + return sourcetransformer.Messages{} + }), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, fmt.Errorf("mock error")) - - ctx, cancel := context.WithTimeout(context.Background(), time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, + conn := newServer(t, func(server *grpc.Server) { + transformpb.RegisterSourceTransformServer(server, svc) }) - }) - - t.Run("test error retryable: failed after 5 retries", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() + transformClient := transformpb.NewSourceTransformClient(conn) + client, _ := sourcetransformerSdk.NewFromClient(transformClient) + u := NewGRPCBasedTransformer("testVertex", client) - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() + // Create a context and cancel it immediately. + // This cancelled context is passed to the ApplyTransform function to simulate failure + ctx, cancel := context.WithCancel(context.Background()) + cancel() - u := NewMockGRPCBasedTransformer(mockClient) _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ Message: isb.Message{ Header: isb.Header{ @@ -241,173 +132,18 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), }}, ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - }) - }) - t.Run("test error retryable: failed after 1 retry", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.InvalidArgument, "mock test err: non retryable").Err()) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ + expectedUDFErr := &rpc.ApplyUDFErr{ UserUDFErr: false, - Message: fmt.Sprintf("%s", err), + Message: "gRPC client.SourceTransformFn failed, Canceled: context canceled", InternalErr: rpc.InternalErr{ Flag: true, MainCarDown: false, }, - }) - }) - - t.Run("test error retryable: success after 1 retry", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169720, 0)), - Watermark: timestamppb.New(time.Time{}), - } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.DeadlineExceeded, "mock test err").Err()) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(&transformpb.SourceTransformResponse{ - Results: []*transformpb.SourceTransformResponse_Result{ - { - Keys: []string{"test_success_key"}, - Value: []byte(`forward_message`), - }, - }, - }, nil) - - ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - got, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169720, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_success_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.NoError(t, err) - assert.Equal(t, req.Keys, got[0].WriteMessages[0].Keys) - assert.Equal(t, req.Value, got[0].WriteMessages[0].Payload) - }) - - t.Run("test error non retryable", func(t *testing.T) { - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - req := &transformpb.SourceTransformRequest{ - Keys: []string{"test_error_key"}, - Value: []byte(`forward_message`), - EventTime: timestamppb.New(time.Unix(1661169660, 0)), - Watermark: timestamppb.New(time.Time{}), } - mockClient.EXPECT().SourceTransformFn(gomock.Any(), &rpcMsg{msg: req}).Return(nil, status.New(codes.InvalidArgument, "mock test err: non retryable").Err()) - ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - _, err := u.ApplyTransform(ctx, []*isb.ReadMessage{{ - Message: isb.Message{ - Header: isb.Header{ - MessageInfo: isb.MessageInfo{ - EventTime: time.Unix(1661169660, 0), - }, - ID: isb.MessageID{ - VertexName: "test-vertex", - Offset: "0-0", - }, - Keys: []string{"test_error_key"}, - }, - Body: isb.Body{ - Payload: []byte(`forward_message`), - }, - }, - ReadOffset: isb.SimpleStringOffset(func() string { return "0" }), - }}, - ) - assert.ErrorIs(t, err, &rpc.ApplyUDFErr{ - UserUDFErr: false, - Message: fmt.Sprintf("%s", err), - InternalErr: rpc.InternalErr{ - Flag: true, - MainCarDown: false, - }, - }) + var receivedErr *rpc.ApplyUDFErr + assert.ErrorAs(t, err, &receivedErr) + assert.Equal(t, expectedUDFErr, receivedErr) }) } @@ -448,114 +184,26 @@ func TestGRPCBasedTransformer_ApplyWithServer_ChangePayload(t *testing.T) { apply, err := u.ApplyTransform(context.TODO(), messages) assert.NoError(t, err) - var results = make([][]byte, len(readMessages)) - var resultKeys = make([][]string, len(readMessages)) - for idx, pair := range apply { - results[idx] = pair.WriteMessages[0].Payload - resultKeys[idx] = pair.WriteMessages[0].Header.Keys - } - - var expectedResults = make([][]byte, count) - var expectedKeys = make([][]string, count) - for idx, readMessage := range readMessages { + for _, pair := range apply { + resultPayload := pair.WriteMessages[0].Payload + resultKeys := pair.WriteMessages[0].Header.Keys var readMessagePayload testutils.PayloadForTest - _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) + _ = json.Unmarshal(pair.ReadMessage.Payload, &readMessagePayload) + var expectedKeys []string if readMessagePayload.Value%2 == 0 { - expectedKeys[idx] = []string{"even"} + expectedKeys = []string{"even"} } else { - expectedKeys[idx] = []string{"odd"} + expectedKeys = []string{"odd"} } + assert.Equal(t, expectedKeys, resultKeys) + doubledValue := testutils.PayloadForTest{ Key: readMessagePayload.Key, Value: readMessagePayload.Value * 2, } marshal, _ := json.Marshal(doubledValue) - expectedResults[idx] = marshal - } - - assert.ElementsMatch(t, expectedResults, results) - assert.Equal(t, expectedKeys, resultKeys) -} - -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangePayload(t *testing.T) { - multiplyBy2 := func(body []byte) interface{} { - var result testutils.PayloadForTest - _ = json.Unmarshal(body, &result) - result.Value = result.Value * 2 - return result - } - - ctrl := gomock.NewController(t) - defer ctrl.Finish() - - mockClient := transformermock.NewMockSourceTransformClient(ctrl) - mockClient.EXPECT().SourceTransformFn(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, datum *transformpb.SourceTransformRequest, opts ...grpc.CallOption) (*transformpb.SourceTransformResponse, error) { - var originalValue testutils.PayloadForTest - _ = json.Unmarshal(datum.GetValue(), &originalValue) - doubledValue, _ := json.Marshal(multiplyBy2(datum.GetValue()).(testutils.PayloadForTest)) - var Results []*transformpb.SourceTransformResponse_Result - if originalValue.Value%2 == 0 { - Results = append(Results, &transformpb.SourceTransformResponse_Result{ - Keys: []string{"even"}, - Value: doubledValue, - }) - } else { - Results = append(Results, &transformpb.SourceTransformResponse_Result{ - Keys: []string{"odd"}, - Value: doubledValue, - }) - } - datumList := &transformpb.SourceTransformResponse{ - Results: Results, - } - return datumList, nil - }, - ).AnyTimes() - - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() - go func() { - <-ctx.Done() - if ctx.Err() == context.DeadlineExceeded { - t.Log(t.Name(), "test timeout") - } - }() - - u := NewMockGRPCBasedTransformer(mockClient) - - var count = int64(10) - readMessages := testutils.BuildTestReadMessages(count, time.Unix(1661169600, 0), nil) - - var results = make([][]byte, len(readMessages)) - var resultKeys = make([][]string, len(readMessages)) - messages := make([]*isb.ReadMessage, len(readMessages)) - for idx, readMessage := range readMessages { - messages[idx] = &readMessage - } - apply, err := u.ApplyTransform(ctx, messages) - assert.NoError(t, err) - for idx, pair := range apply { - results[idx] = pair.WriteMessages[0].Payload - resultKeys[idx] = pair.WriteMessages[0].Header.Keys - } - - var expectedResults = make([][]byte, count) - var expectedKeys = make([][]string, count) - for idx, readMessage := range readMessages { - var readMessagePayload testutils.PayloadForTest - _ = json.Unmarshal(readMessage.Payload, &readMessagePayload) - if readMessagePayload.Value%2 == 0 { - expectedKeys[idx] = []string{"even"} - } else { - expectedKeys[idx] = []string{"odd"} - } - marshal, _ := json.Marshal(multiplyBy2(readMessage.Payload)) - expectedResults[idx] = marshal + assert.Equal(t, marshal, resultPayload) } - - assert.Equal(t, expectedResults, results) - assert.Equal(t, expectedKeys, resultKeys) } func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientConn { @@ -609,7 +257,7 @@ func newServer(t *testing.T, register func(server *grpc.Server)) *grpc.ClientCon return conn } -func TestGRPCBasedTransformer_ApplyWithMockClient_ChangeEventTime(t *testing.T) { +func TestGRPCBasedTransformer_Apply_ChangeEventTime(t *testing.T) { testEventTime := time.Date(1992, 2, 8, 0, 0, 0, 100, time.UTC) svc := &sourcetransformer.Service{ Transformer: sourcetransformer.SourceTransformFunc(func(ctx context.Context, keys []string, datum sourcetransformer.Datum) sourcetransformer.Messages { From 56001a90d34d4322677669459a108644bf3d0644 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Wed, 18 Sep 2024 18:35:34 +0530 Subject: [PATCH 06/15] Code generation Signed-off-by: Sreekanth --- pkg/sources/forward/data_forward_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index e4fb00a2e0..96cb6760e6 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -883,7 +883,7 @@ func (f mySourceForwardTest) ApplyTransform(ctx context.Context, messages []*isb } results[i] = isb.ReadWriteMessagePair{ ReadMessage: message, - WriteMessages: []*isb.WriteMessage{&isb.WriteMessage{ + WriteMessages: []*isb.WriteMessage{{ Message: writeMsg, }}, } From 57a03948ce5da646ae54cb5ae049b0f0e65db804 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 14:56:47 +0530 Subject: [PATCH 07/15] Go mod tidy Signed-off-by: Sreekanth --- go.sum | 36 ++++++++++++++++++------------------ 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/go.sum b/go.sum index 5ebfdd5ebe..7dfca21927 100644 --- a/go.sum +++ b/go.sum @@ -446,8 +446,8 @@ github.com/mattn/go-runewidth v0.0.8/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m github.com/mattn/go-runewidth v0.0.10/go.mod h1:RAqKPSqVFrSLVXbA8x7dzmKdmGzieGRCM46jaSJTDAk= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0 h1:jWpvCLoY8Z/e3VKvlsiIGKtc+UG6U5vzxaoagmhXfyg= github.com/matttproud/golang_protobuf_extensions/v2 v2.0.0/go.mod h1:QUyp042oQthUoa9bqDv0ER0wrtXnBruoNd7aNjkbP+k= -github.com/minio/highwayhash v1.0.3 h1:kbnuUMoHYyVl7szWjSxJnxw11k2U709jqFPPmIUyD6Q= -github.com/minio/highwayhash v1.0.3/go.mod h1:GGYsuwP/fPD6Y9hMiXuapVvlIUEhFhMTh0rxU3ik1LQ= +github.com/minio/highwayhash v1.0.2 h1:Aak5U0nElisjDCfPSG79Tgzkn2gl66NxOMspRrKnA/g= +github.com/minio/highwayhash v1.0.2/go.mod h1:BQskDq+xkJ12lmlUUi7U0M5Swg3EWR+dLTk+kldvVxY= github.com/mitchellh/copystructure v1.0.0/go.mod h1:SNtv71yrdKgLRyLFxmLdkAbkKEFWgYaq1OVrnRcwhnw= github.com/mitchellh/copystructure v1.2.0 h1:vpKXTN4ewci03Vljg/q9QvCGUDttBOGBIa15WveJJGw= github.com/mitchellh/copystructure v1.2.0/go.mod h1:qLl+cE2AmVv+CoeAwDPye/v+N2HKCj9FbZEVFJRxO9s= @@ -474,12 +474,12 @@ github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f h1:y5//uYreIhSUg3J1GEMiLbxo1LJaP8RfCpH6pymGZus= github.com/mxk/go-flowrate v0.0.0-20140419014527-cca7078d478f/go.mod h1:ZdcZmHo+o7JKHSa8/e818NopupXU1YMK5fe1lsApnBw= -github.com/nats-io/jwt/v2 v2.5.8 h1:uvdSzwWiEGWGXf+0Q+70qv6AQdvcvxrv9hPM0RiPamE= -github.com/nats-io/jwt/v2 v2.5.8/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= -github.com/nats-io/nats-server/v2 v2.10.20 h1:CXDTYNHeBiAKBTAIP2gjpgbWap2GhATnTLgP8etyvEI= -github.com/nats-io/nats-server/v2 v2.10.20/go.mod h1:hgcPnoUtMfxz1qVOvLZGurVypQ+Cg6GXVXjG53iHk+M= -github.com/nats-io/nats.go v1.37.0 h1:07rauXbVnnJvv1gfIyghFEo6lUcYRY0WXc3x7x0vUxE= -github.com/nats-io/nats.go v1.37.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= +github.com/nats-io/jwt/v2 v2.5.7 h1:j5lH1fUXCnJnY8SsQeB/a/z9Azgu2bYIDvtPVNdxe2c= +github.com/nats-io/jwt/v2 v2.5.7/go.mod h1:ZdWS1nZa6WMZfFwwgpEaqBV8EPGVgOTDHN/wTbz0Y5A= +github.com/nats-io/nats-server/v2 v2.10.17 h1:PTVObNBD3TZSNUDgzFb1qQsQX4mOgFmOuG9vhT+KBUY= +github.com/nats-io/nats-server/v2 v2.10.17/go.mod h1:5OUyc4zg42s/p2i92zbbqXvUNsbF0ivdTLKshVMn2YQ= +github.com/nats-io/nats.go v1.36.0 h1:suEUPuWzTSse/XhESwqLxXGuj8vGRuPRoG7MoRN/qyU= +github.com/nats-io/nats.go v1.36.0/go.mod h1:Ubdu4Nh9exXdSz0RVWRFBbRfrbSxOYd26oF0wkWclB8= github.com/nats-io/nkeys v0.4.7 h1:RwNJbbIdYCoClSDNY7QVKZlyb/wfT6ugvFCiKy6vDvI= github.com/nats-io/nkeys v0.4.7/go.mod h1:kqXRgRDPlGy7nGaEDMuYzmiJCIAAWDK0IMBtDmGD0nc= github.com/nats-io/nuid v1.0.1 h1:5iA8DT8V7q8WK2EScv2padNa/rTESc1KdnPw4TC2paw= @@ -687,8 +687,8 @@ golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5y golang.org/x/crypto v0.0.0-20220214200702-86341886e292/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= golang.org/x/crypto v0.3.0/go.mod h1:hebNnKkNXi2UzZN1eVRvBB7co0a+JxK6XbPiWVs/3J4= golang.org/x/crypto v0.6.0/go.mod h1:OFC/31mSvZgRz0V1QTNCzfAI1aIRzbiufJtkMIlEp58= -golang.org/x/crypto v0.26.0 h1:RrRspgV4mU+YwB4FYnuBoKsUapNIL5cohGAmSH3azsw= -golang.org/x/crypto v0.26.0/go.mod h1:GY7jblb9wI+FOo5y8/S2oY4zWP07AkOJ4+jxCqdqn54= +golang.org/x/crypto v0.24.0 h1:mnl8DM0o513X8fdIkmyFE/5hTYxbwYOjDS/+rK6qpRI= +golang.org/x/crypto v0.24.0/go.mod h1:Z1PMYSOR5nyMcyAVAIQSKCDwalqy85Aqn1x3Ws4L5DM= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190306152737-a1d7652674e8/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/exp v0.0.0-20190510132918-efd6b22b2522/go.mod h1:ZjyILWgesfNpC6sMxTJOJm9Kp84zZh5NQWvqDGG3Qr8= @@ -806,6 +806,7 @@ golang.org/x/sync v0.8.0 h1:3NFvSEYkUoMifnESzZl15y791HH1qU2xm6eCJU5ZPXQ= golang.org/x/sync v0.8.0/go.mod h1:Czt+wKu1gCyEFDUtn0jG5QVvpJ6rzVqr5aXyt9drQfk= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190130150945-aca44879d564/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190312061237-fead79001313/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -862,16 +863,15 @@ golang.org/x/sys v0.0.0-20220811171246-fbc7d0a398ab/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.2.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.6.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.21.0 h1:rF+pYz3DAGSQAxAu1CbC7catZg4ebC4UIeIhKxBZvws= golang.org/x/sys v0.21.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= -golang.org/x/sys v0.24.0 h1:Twjiwq9dn6R1fQcyiK+wQyHWfaz/BJB+YIpzU/Cv3Xg= -golang.org/x/sys v0.24.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210615171337-6886f2dfbf5b/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.2.0/go.mod h1:TVmDHMZPmdnySmBfhjOoOdhjzdE1h4u1VwSiw2l1Nuc= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= -golang.org/x/term v0.23.0 h1:F6D4vR+EHoL9/sWAWgAR1H2DcHr4PareCbAaCo1RpuU= -golang.org/x/term v0.23.0/go.mod h1:DgV24QBUrK6jhZXl+20l6UWznPlwAHm1Q1mGHtydmSk= +golang.org/x/term v0.21.0 h1:WVXCp+/EBEHOj53Rvu+7KiT/iElMrO8ACK16SMZ3jaA= +golang.org/x/term v0.21.0/go.mod h1:ooXLefLobQVslOqselCNF4SxFAaoS6KujMbsGzSDmX0= golang.org/x/text v0.0.0-20170915032832-14c0d48ead0c/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= @@ -884,14 +884,14 @@ golang.org/x/text v0.3.7/go.mod h1:u+2+/6zg+i71rQMx5EYifcz6MCKuco9NR6JIITiCfzQ= golang.org/x/text v0.3.8/go.mod h1:E6s5w1FMmriuDzIBO73fBruAKo1PCIq6d2Q6DHfQ8WQ= golang.org/x/text v0.4.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= golang.org/x/text v0.7.0/go.mod h1:mrYo+phRRbMaCq/xk9113O4dZlRixOauAjOtrjsXDZ8= -golang.org/x/text v0.17.0 h1:XtiM5bkSOt+ewxlOE/aE/AKEHibwj/6gvWMl9Rsh0Qc= -golang.org/x/text v0.17.0/go.mod h1:BuEKDfySbSR4drPmRPG/7iBdf8hvFMuRexcpahXilzY= +golang.org/x/text v0.16.0 h1:a94ExnEXNtEwYLGJSIUxnWoxoRz/ZcCsV63ROupILh4= +golang.org/x/text v0.16.0/go.mod h1:GhwF1Be+LQoKShO3cGOHzqOgRrGaYc9AvblQOmPVHnI= golang.org/x/time v0.0.0-20181108054448-85acf8d2951c/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20190308202827-9d24e82272b4/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20191024005414-555d28b269f0/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac/go.mod h1:tRJNPiyCQ0inRvYxbN9jk5I+vvW/OXSQhTDSoE431IQ= -golang.org/x/time v0.6.0 h1:eTDhh4ZXt5Qf0augr54TN6suAUudPcawVZeIAPU7D4U= -golang.org/x/time v0.6.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= +golang.org/x/time v0.5.0 h1:o7cqy6amK/52YcAKIPlM3a+Fpj35zvRj2TP+e1xFSfk= +golang.org/x/time v0.5.0/go.mod h1:3BpzKBy/shNhVucY/MWOyx10tF3SFh9QdLuxbVysPQM= golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= From 302b11e47f570081174f8dadc5e33a14d236f13d Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 15:56:24 +0530 Subject: [PATCH 08/15] Debug test failure Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 2 ++ test/e2e/functional_test.go | 2 +- test/fixtures/when.go | 2 +- 3 files changed, 4 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 9eaa4de89b..38281a57ac 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -244,3 +244,5 @@ jobs: env: GOPATH: /home/runner/go run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} + - name: Debug + run: KUBECONFIG=~/.kube/numaflow-e2e-config kubectl -n numaflow-system describe deployments diff --git a/test/e2e/functional_test.go b/test/e2e/functional_test.go index 1c72af9229..815874b6ad 100644 --- a/test/e2e/functional_test.go +++ b/test/e2e/functional_test.go @@ -41,7 +41,7 @@ func (s *FunctionalSuite) TestCreateSimplePipeline() { w := s.Given().Pipeline("@testdata/simple-pipeline.yaml"). When(). CreatePipelineAndWait() - defer w.DeletePipelineAndWait() + //defer w.DeletePipelineAndWait() pipelineName := "simple-pipeline" w.Expect(). diff --git a/test/fixtures/when.go b/test/fixtures/when.go index 986085bdd6..2bfd2a470d 100644 --- a/test/fixtures/when.go +++ b/test/fixtures/when.go @@ -98,7 +98,7 @@ func (w *When) CreatePipelineAndWait() *When { } // wait if err := WaitForPipelineRunning(ctx, w.pipelineClient, w.pipeline.Name, defaultTimeout); err != nil { - w.t.Fatal(err) + w.t.Log("Waiting for pipeline:", err) } return w } From 86e6032da473e0a0b52ebd2d9d15881f2b42198f Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 16:24:36 +0530 Subject: [PATCH 09/15] Debug: Run debug steps on failure Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 38281a57ac..910060fd40 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -243,6 +243,4 @@ jobs: - name: Run tests env: GOPATH: /home/runner/go - run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} - - name: Debug - run: KUBECONFIG=~/.kube/numaflow-e2e-config kubectl -n numaflow-system describe deployments + run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system describe deployments && exit 1 From 7d4b8dfefeff9dda003c35ffffb00dec29e78a87 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 17:42:39 +0530 Subject: [PATCH 10/15] Debug: disable cleanup Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 2 +- Makefile | 2 +- test/e2e/functional_test.go | 4 ++++ test/fixtures/e2e_suite.go | 1 + 4 files changed, 7 insertions(+), 2 deletions(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 910060fd40..1898abade2 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -243,4 +243,4 @@ jobs: - name: Run tests env: GOPATH: /home/runner/go - run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system describe deployments && exit 1 + run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system get deployments && kubectl -n numaflow-system describe deploy -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system describe pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system logs -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' diff --git a/Makefile b/Makefile index a4bc2012bc..e88d460b5c 100644 --- a/Makefile +++ b/Makefile @@ -136,7 +136,7 @@ endif cat test/manifests/e2e-api-pod.yaml | sed 's@quay.io/numaproj/@$(IMAGE_NAMESPACE)/@' | sed 's/:latest/:$(VERSION)/' | kubectl -n numaflow-system apply -f - go generate $(shell find ./test/$* -name '*.go') go test -v -timeout 15m -count 1 --tags test -p 1 ./test/$* - $(MAKE) cleanup-e2e + # $(MAKE) cleanup-e2e image-restart: $(MAKE) image diff --git a/test/e2e/functional_test.go b/test/e2e/functional_test.go index 815874b6ad..8040f85189 100644 --- a/test/e2e/functional_test.go +++ b/test/e2e/functional_test.go @@ -149,6 +149,7 @@ func (s *FunctionalSuite) TestCreateSimplePipeline() { } func (s *FunctionalSuite) TestUDFFiltering() { + return w := s.Given().Pipeline("@testdata/udf-filtering.yaml"). When(). CreatePipelineAndWait() @@ -178,6 +179,7 @@ func (s *FunctionalSuite) TestUDFFiltering() { } func (s *FunctionalSuite) TestDropOnFull() { + return // the drop on full feature is not supported with redis ISBSVC if strings.ToUpper(os.Getenv("ISBSVC")) == "REDIS" { @@ -228,6 +230,7 @@ func (s *FunctionalSuite) TestDropOnFull() { } func (s *FunctionalSuite) TestWatermarkEnabled() { + return // the watermark feature is not supported with redis ISBSVC if strings.ToUpper(os.Getenv("ISBSVC")) == "REDIS" { @@ -309,6 +312,7 @@ func isWatermarkProgressing(ctx context.Context, client daemonclient.DaemonClien } func (s *FunctionalSuite) TestFallbackSink() { + return w := s.Given().Pipeline("@testdata/simple-fallback.yaml"). When(). diff --git a/test/fixtures/e2e_suite.go b/test/fixtures/e2e_suite.go index be1255a037..a3851a97f4 100644 --- a/test/fixtures/e2e_suite.go +++ b/test/fixtures/e2e_suite.go @@ -132,6 +132,7 @@ func (s *E2ESuite) SetupSuite() { } func (s *E2ESuite) TearDownSuite() { + return s.deleteResources([]schema.GroupVersionResource{ dfv1.PipelineGroupVersionResource, }) From d3f750a28c57dd47d8aa0f59a4db34f8f56805a1 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 21:00:21 +0530 Subject: [PATCH 11/15] Fix for Go sdk version Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 4 ++++ pkg/sdkclient/serverinfo/types.go | 2 +- 2 files changed, 5 insertions(+), 1 deletion(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 1898abade2..f45664d8e4 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -244,3 +244,7 @@ jobs: env: GOPATH: /home/runner/go run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system get deployments && kubectl -n numaflow-system describe deploy -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system describe pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system logs -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' + - name: Debug + env: + KUBECONFIG: /home/runner/.kube/numaflow-e2e-config + run: kubectl -n numaflow-system get deployments && kubectl -n numaflow-system describe deploy -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system get pod && kubectl -n numaflow-system get pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system describe pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system logs -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' diff --git a/pkg/sdkclient/serverinfo/types.go b/pkg/sdkclient/serverinfo/types.go index fc8fdd9b81..334a9f4a8e 100644 --- a/pkg/sdkclient/serverinfo/types.go +++ b/pkg/sdkclient/serverinfo/types.go @@ -28,7 +28,7 @@ const ( type sdkConstraints map[Language]string var minimumSupportedSDKVersions = sdkConstraints{ - Go: "0.8.0", + Go: "0.8.0-0", Python: "0.8.0", Java: "0.8.0", Rust: "0.1.0", From 148f308ec51ac86b2decc2aab123eb8bb33e2a04 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Thu, 19 Sep 2024 21:48:39 +0530 Subject: [PATCH 12/15] Revert changes made for debugging Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 6 +----- Makefile | 2 +- test/e2e/functional_test.go | 7 ++----- test/fixtures/e2e_suite.go | 2 +- test/fixtures/when.go | 2 +- 5 files changed, 6 insertions(+), 13 deletions(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index f45664d8e4..9eaa4de89b 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -243,8 +243,4 @@ jobs: - name: Run tests env: GOPATH: /home/runner/go - run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system get deployments && kubectl -n numaflow-system describe deploy -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system describe pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system logs -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' - - name: Debug - env: - KUBECONFIG: /home/runner/.kube/numaflow-e2e-config - run: kubectl -n numaflow-system get deployments && kubectl -n numaflow-system describe deploy -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system get pod && kubectl -n numaflow-system get pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system describe pod -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' && kubectl -n numaflow-system logs -l 'numaflow.numaproj.io/pipeline-name=simple-pipeline' + run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} diff --git a/Makefile b/Makefile index e88d460b5c..a4bc2012bc 100644 --- a/Makefile +++ b/Makefile @@ -136,7 +136,7 @@ endif cat test/manifests/e2e-api-pod.yaml | sed 's@quay.io/numaproj/@$(IMAGE_NAMESPACE)/@' | sed 's/:latest/:$(VERSION)/' | kubectl -n numaflow-system apply -f - go generate $(shell find ./test/$* -name '*.go') go test -v -timeout 15m -count 1 --tags test -p 1 ./test/$* - # $(MAKE) cleanup-e2e + $(MAKE) cleanup-e2e image-restart: $(MAKE) image diff --git a/test/e2e/functional_test.go b/test/e2e/functional_test.go index 8040f85189..582fc37a3a 100644 --- a/test/e2e/functional_test.go +++ b/test/e2e/functional_test.go @@ -41,7 +41,7 @@ func (s *FunctionalSuite) TestCreateSimplePipeline() { w := s.Given().Pipeline("@testdata/simple-pipeline.yaml"). When(). CreatePipelineAndWait() - //defer w.DeletePipelineAndWait() + defer w.DeletePipelineAndWait() pipelineName := "simple-pipeline" w.Expect(). @@ -149,7 +149,7 @@ func (s *FunctionalSuite) TestCreateSimplePipeline() { } func (s *FunctionalSuite) TestUDFFiltering() { - return + w := s.Given().Pipeline("@testdata/udf-filtering.yaml"). When(). CreatePipelineAndWait() @@ -179,7 +179,6 @@ func (s *FunctionalSuite) TestUDFFiltering() { } func (s *FunctionalSuite) TestDropOnFull() { - return // the drop on full feature is not supported with redis ISBSVC if strings.ToUpper(os.Getenv("ISBSVC")) == "REDIS" { @@ -230,7 +229,6 @@ func (s *FunctionalSuite) TestDropOnFull() { } func (s *FunctionalSuite) TestWatermarkEnabled() { - return // the watermark feature is not supported with redis ISBSVC if strings.ToUpper(os.Getenv("ISBSVC")) == "REDIS" { @@ -312,7 +310,6 @@ func isWatermarkProgressing(ctx context.Context, client daemonclient.DaemonClien } func (s *FunctionalSuite) TestFallbackSink() { - return w := s.Given().Pipeline("@testdata/simple-fallback.yaml"). When(). diff --git a/test/fixtures/e2e_suite.go b/test/fixtures/e2e_suite.go index a3851a97f4..cd0917d3e0 100644 --- a/test/fixtures/e2e_suite.go +++ b/test/fixtures/e2e_suite.go @@ -132,7 +132,7 @@ func (s *E2ESuite) SetupSuite() { } func (s *E2ESuite) TearDownSuite() { - return + s.deleteResources([]schema.GroupVersionResource{ dfv1.PipelineGroupVersionResource, }) diff --git a/test/fixtures/when.go b/test/fixtures/when.go index 2bfd2a470d..986085bdd6 100644 --- a/test/fixtures/when.go +++ b/test/fixtures/when.go @@ -98,7 +98,7 @@ func (w *When) CreatePipelineAndWait() *When { } // wait if err := WaitForPipelineRunning(ctx, w.pipelineClient, w.pipeline.Name, defaultTimeout); err != nil { - w.t.Log("Waiting for pipeline:", err) + w.t.Fatal(err) } return w } From 61b9d2450f07e5578135fed3006ab5001e127d18 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Fri, 20 Sep 2024 10:13:05 +0530 Subject: [PATCH 13/15] Disable source transformer tests with python,java,rust. Upgrade golangci-lint Signed-off-by: Sreekanth --- Makefile | 2 +- .../sourcetransformer/client_test.go | 9 ++++--- .../transformer/grpc_transformer_test.go | 7 +++--- test/transformer-e2e/transformer_test.go | 25 ++++++++++--------- 4 files changed, 23 insertions(+), 20 deletions(-) diff --git a/Makefile b/Makefile index a4bc2012bc..11d91c5890 100644 --- a/Makefile +++ b/Makefile @@ -244,7 +244,7 @@ manifests: crds kubectl kustomize config/extensions/webhook > config/validating-webhook-install.yaml $(GOPATH)/bin/golangci-lint: - curl -sSfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh | sh -s -- -b `go env GOPATH`/bin v1.54.1 + curl -sSfL https://raw.githubusercontent.com/golangci/golangci-lint/master/install.sh | sh -s -- -b `go env GOPATH`/bin v1.61.0 .PHONY: lint lint: $(GOPATH)/bin/golangci-lint diff --git a/pkg/sdkclient/sourcetransformer/client_test.go b/pkg/sdkclient/sourcetransformer/client_test.go index b528b037bc..86f3cb9e59 100644 --- a/pkg/sdkclient/sourcetransformer/client_test.go +++ b/pkg/sdkclient/sourcetransformer/client_test.go @@ -20,6 +20,11 @@ import ( "context" "errors" "fmt" + "net" + "reflect" + "testing" + "time" + "github.com/golang/mock/gomock" transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" transformermock "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1/transformmock" @@ -30,10 +35,6 @@ import ( "google.golang.org/grpc/test/bufconn" "google.golang.org/protobuf/types/known/emptypb" "google.golang.org/protobuf/types/known/timestamppb" - "net" - "reflect" - "testing" - "time" ) func TestClient_IsReady(t *testing.T) { diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index 600369b255..2eb58cd3c6 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -20,13 +20,14 @@ import ( "context" "encoding/json" "errors" - "github.com/numaproj/numaflow-go/pkg/sourcetransformer" - "google.golang.org/grpc/credentials/insecure" - "google.golang.org/grpc/test/bufconn" "net" "testing" "time" + "github.com/numaproj/numaflow-go/pkg/sourcetransformer" + "google.golang.org/grpc/credentials/insecure" + "google.golang.org/grpc/test/bufconn" + transformpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sourcetransform/v1" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" diff --git a/test/transformer-e2e/transformer_test.go b/test/transformer-e2e/transformer_test.go index 55b88f3683..77fae76814 100644 --- a/test/transformer-e2e/transformer_test.go +++ b/test/transformer-e2e/transformer_test.go @@ -174,22 +174,23 @@ func (s *TransformerSuite) TestSourceTransformer() { var wg sync.WaitGroup wg.Add(4) - go func() { - defer wg.Done() - s.testSourceTransformer("python") - }() - go func() { - defer wg.Done() - s.testSourceTransformer("java") - }() + // FIXME: Enable these tests after corresponding SDKs are changed to support bidirectional streaming + //go func() { + // defer wg.Done() + // s.testSourceTransformer("python") + //}() + //go func() { + // defer wg.Done() + // s.testSourceTransformer("java") + //}() go func() { defer wg.Done() s.testSourceTransformer("go") }() - go func() { - defer wg.Done() - s.testSourceTransformer("rust") - }() + //go func() { + // defer wg.Done() + // s.testSourceTransformer("rust") + //}() wg.Wait() } From 7016796c1b16be7c0debd184f28d61c7d6402606 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Fri, 20 Sep 2024 13:20:59 +0530 Subject: [PATCH 14/15] Fix golangci-lint errors Signed-off-by: Sreekanth --- pkg/udf/forward/forward.go | 2 +- pkg/webhook/validator/validator.go | 5 ++++- test/transformer-e2e/transformer_test.go | 2 +- 3 files changed, 6 insertions(+), 3 deletions(-) diff --git a/pkg/udf/forward/forward.go b/pkg/udf/forward/forward.go index 53efc945da..e768808cc3 100644 --- a/pkg/udf/forward/forward.go +++ b/pkg/udf/forward/forward.go @@ -481,7 +481,7 @@ func (isdf *InterStepDataForward) streamMessage(ctx context.Context, dataMessage if len(dataMessages) > 1 { errMsg := "data message size is not 1 with map UDF streaming" isdf.opts.logger.Errorw(errMsg) - return nil, fmt.Errorf(errMsg) + return nil, errors.New(errMsg) } else if len(dataMessages) == 1 { // send to map UDF only the data messages diff --git a/pkg/webhook/validator/validator.go b/pkg/webhook/validator/validator.go index d5f2e86664..6d4e3e46a1 100644 --- a/pkg/webhook/validator/validator.go +++ b/pkg/webhook/validator/validator.go @@ -83,7 +83,10 @@ func GetValidator(ctx context.Context, NumaClient v1alpha1.NumaflowV1alpha1Inter // DeniedResponse constructs a denied AdmissionResponse func DeniedResponse(reason string, args ...interface{}) *admissionv1.AdmissionResponse { - result := apierrors.NewBadRequest(fmt.Sprintf(reason, args...)).Status() + if len(args) > 0 { + reason = fmt.Sprintf(reason, args) + } + result := apierrors.NewBadRequest(reason).Status() return &admissionv1.AdmissionResponse{ Result: &result, Allowed: false, diff --git a/test/transformer-e2e/transformer_test.go b/test/transformer-e2e/transformer_test.go index 77fae76814..8c07db744c 100644 --- a/test/transformer-e2e/transformer_test.go +++ b/test/transformer-e2e/transformer_test.go @@ -173,7 +173,7 @@ func (s *TransformerSuite) TestSourceTransformer() { } var wg sync.WaitGroup - wg.Add(4) + wg.Add(1) // FIXME: Enable these tests after corresponding SDKs are changed to support bidirectional streaming //go func() { // defer wg.Done() From 1822491bff6a62e036fe8129d0c81a51b421a125 Mon Sep 17 00:00:00 2001 From: Sreekanth Date: Mon, 23 Sep 2024 09:39:18 +0530 Subject: [PATCH 15/15] Debug: show pod logs Signed-off-by: Sreekanth --- .github/workflows/ci.yaml | 2 +- Makefile | 2 +- test/transformer-e2e/transformer_test.go | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/.github/workflows/ci.yaml b/.github/workflows/ci.yaml index 9eaa4de89b..d236e150d4 100644 --- a/.github/workflows/ci.yaml +++ b/.github/workflows/ci.yaml @@ -243,4 +243,4 @@ jobs: - name: Run tests env: GOPATH: /home/runner/go - run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} + run: KUBECONFIG=~/.kube/numaflow-e2e-config VERSION=${{ github.sha }} ISBSVC=${{matrix.driver}} SKIP_IMAGE_BUILD=true make test-${{matrix.case}} || kubectl -n numaflow-system get pods && kubectl -n numaflow-system get pods -l 'numaflow.numaproj.io/vertex-name=in' && kubectl -n numaflow-system describe pods -l 'numaflow.numaproj.io/vertex-name=in' && kubectl -n numaflow-system logs -c numa -l 'numaflow.numaproj.io/vertex-name=in' && kubectl -n numaflow-system logs -c transformer -l 'numaflow.numaproj.io/vertex-name=in' diff --git a/Makefile b/Makefile index 11d91c5890..5a56fba63d 100644 --- a/Makefile +++ b/Makefile @@ -136,7 +136,7 @@ endif cat test/manifests/e2e-api-pod.yaml | sed 's@quay.io/numaproj/@$(IMAGE_NAMESPACE)/@' | sed 's/:latest/:$(VERSION)/' | kubectl -n numaflow-system apply -f - go generate $(shell find ./test/$* -name '*.go') go test -v -timeout 15m -count 1 --tags test -p 1 ./test/$* - $(MAKE) cleanup-e2e + #$(MAKE) cleanup-e2e image-restart: $(MAKE) image diff --git a/test/transformer-e2e/transformer_test.go b/test/transformer-e2e/transformer_test.go index 8c07db744c..3b0fe3d89a 100644 --- a/test/transformer-e2e/transformer_test.go +++ b/test/transformer-e2e/transformer_test.go @@ -198,7 +198,7 @@ func (s *TransformerSuite) testSourceTransformer(lang string) { w := s.Given().Pipeline(fmt.Sprintf("@testdata/event-time-filter-%s.yaml", lang)). When(). CreatePipelineAndWait() - defer w.DeletePipelineAndWait() + //defer w.DeletePipelineAndWait() pipelineName := fmt.Sprintf("event-time-filter-%s", lang) // wait for all the pods to come up