diff --git a/examples/1-simple-pipeline-keys.yaml b/examples/1-simple-pipeline-keys.yaml index ae7b80f717..cfdf84d37d 100644 --- a/examples/1-simple-pipeline-keys.yaml +++ b/examples/1-simple-pipeline-keys.yaml @@ -24,4 +24,4 @@ spec: - from: in to: cat - from: cat - to: out + to: out \ No newline at end of file diff --git a/examples/1-simple-pipeline.yaml b/examples/1-simple-pipeline.yaml index fb7e5f215b..e9a304c1a5 100644 --- a/examples/1-simple-pipeline.yaml +++ b/examples/1-simple-pipeline.yaml @@ -22,4 +22,4 @@ spec: - from: in to: cat - from: cat - to: out \ No newline at end of file + to: out diff --git a/pkg/apis/numaflow/v1alpha1/const.go b/pkg/apis/numaflow/v1alpha1/const.go index f129efdf53..bea7e8f40f 100644 --- a/pkg/apis/numaflow/v1alpha1/const.go +++ b/pkg/apis/numaflow/v1alpha1/const.go @@ -42,8 +42,9 @@ const ( RemovePauseTimestampPatch = `[{"op": "remove", "path": "/metadata/annotations/numaflow.numaproj.io~1pause-timestamp"}]` // ID key in the header of sources like http - KeyMetaID = "x-numaflow-id" - KeyMetaEventTime = "x-numaflow-event-time" + KeyMetaID = "X-Numaflow-Id" + KeyMetaEventTime = "X-Numaflow-Event-Time" + KeyMetaCallbackURL = "X-Numaflow-Callback-Url" DefaultISBSvcName = "default" @@ -99,6 +100,9 @@ const ( EnvNamespace = "NUMAFLOW_NAMESPACE" EnvPipelineName = "NUMAFLOW_PIPELINE_NAME" EnvVertexName = "NUMAFLOW_VERTEX_NAME" + EnvMapStreaming = "NUMAFLOW_MAP_STREAMING" + EnvCallbackEnabled = "NUMAFLOW_CALLBACK_ENABLED" + EnvCallbackURL = "NUMAFLOW_CALLBACK_URL" EnvPod = "NUMAFLOW_POD" EnvReplica = "NUMAFLOW_REPLICA" EnvVertexObject = "NUMAFLOW_VERTEX_OBJECT" @@ -209,6 +213,10 @@ const ( PipelineStatusInactive = "inactive" PipelineStatusDeleting = "deleting" PipelineStatusUnhealthy = "unhealthy" + + // Callback annotation keys + CallbackEnabledKey = "numaflow.numaproj.io/callback" + CallbackURLKey = "numaflow.numaproj.io/callback-url" ) var ( diff --git a/pkg/apis/numaflow/v1alpha1/vertex_types.go b/pkg/apis/numaflow/v1alpha1/vertex_types.go index 74663cec72..774e538e16 100644 --- a/pkg/apis/numaflow/v1alpha1/vertex_types.go +++ b/pkg/apis/numaflow/v1alpha1/vertex_types.go @@ -20,7 +20,6 @@ import ( "errors" "fmt" "os" - "strconv" "strings" corev1 "k8s.io/api/core/v1" @@ -176,6 +175,9 @@ func (v Vertex) commonEnvs() []corev1.EnvVar { {Name: EnvReplica, ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{FieldPath: "metadata.annotations['" + KeyReplica + "']"}}}, {Name: EnvPipelineName, Value: v.Spec.PipelineName}, {Name: EnvVertexName, Value: v.Spec.Name}, + {Name: EnvMapStreaming, ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{FieldPath: "metadata.annotations['" + MapUdfStreamKey + "']"}}}, + {Name: EnvCallbackEnabled, ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{FieldPath: "metadata.annotations['" + CallbackEnabledKey + "']"}}}, + {Name: EnvCallbackURL, ValueFrom: &corev1.EnvVarSource{FieldRef: &corev1.ObjectFieldSelector{FieldPath: "metadata.annotations['" + CallbackURLKey + "']"}}}, } } @@ -412,15 +414,6 @@ func (v Vertex) GetReplicas() int { return int(*v.Spec.Replicas) } -func (v Vertex) MapUdfStreamEnabled() (bool, error) { - if v.Spec.Metadata != nil && v.Spec.Metadata.Annotations != nil { - if mapUdfStream, existing := v.Spec.Metadata.Annotations[MapUdfStreamKey]; existing { - return strconv.ParseBool(mapUdfStream) - } - } - return false, nil -} - type VertexSpec struct { AbstractVertex `json:",inline" protobuf:"bytes,1,opt,name=abstractVertex"` PipelineName string `json:"pipelineName" protobuf:"bytes,2,opt,name=pipelineName"` diff --git a/pkg/isb/message.go b/pkg/isb/message.go index fc475ccd7e..5c65117627 100644 --- a/pkg/isb/message.go +++ b/pkg/isb/message.go @@ -17,6 +17,7 @@ limitations under the License. package isb import ( + "fmt" "time" ) @@ -63,8 +64,8 @@ type Header struct { MessageInfo // Kind indicates the kind of Message Kind MessageKind - // ID is used for exactly-once-semantics. ID is usually populated from the offset, if offset is available. - ID string + // ID is used for exactly-once-semantics. ID is a combination of vertex name, offset and index of the message. + ID MessageID // Keys is (key,value) in the map-reduce paradigm will be used for reduce operation, last key in the list // will be used for conditional forwarding Keys []string @@ -73,6 +74,23 @@ type Header struct { Headers map[string]string } +// MessageID is the message ID of the message which is used for exactly-once-semantics. +type MessageID struct { + // VertexName is the name of the vertex + VertexName string + // Offset is the offset of the message + // NOTE: should be unique across the replicas of the vertex, that is the + // reason we don't have a separate replica field in the MessageID + Offset string + // Index is the index of a flatmap message, otherwise use 0 + Index int32 +} + +// String returns the string representation of the MessageID +func (id MessageID) String() string { + return fmt.Sprintf("%s-%s-%d", id.VertexName, id.Offset, id.Index) +} + // Body is the body of the message type Body struct { Payload []byte @@ -104,3 +122,12 @@ type WriteMessage struct { Message Tags []string } + +// ReadWriteMessagePair is a pair of ReadMessage and a list of WriteMessage which will be used +// to map the read message to a list of write messages that the udf returns. +// The error field is used to capture any error that occurs during the processing of the message. +type ReadWriteMessagePair struct { + ReadMessage *ReadMessage + WriteMessages []*WriteMessage + Err error +} diff --git a/pkg/isb/serde.go b/pkg/isb/serde.go index d62b168fea..9f6de7253b 100644 --- a/pkg/isb/serde.go +++ b/pkg/isb/serde.go @@ -54,6 +54,53 @@ func (p *MessageInfo) UnmarshalBinary(data []byte) (err error) { return nil } +type messageIDPreamble struct { + VertexNameLen int16 + OffsetLen int32 + Index int32 +} + +// MarshalBinary encodes MessageID to the binary format +func (id MessageID) MarshalBinary() (data []byte, err error) { + var buf = new(bytes.Buffer) + var preamble = messageIDPreamble{ + VertexNameLen: int16(len(id.VertexName)), + OffsetLen: int32(len(id.Offset)), + Index: id.Index, + } + if err = binary.Write(buf, binary.LittleEndian, preamble); err != nil { + return nil, err + } + if err = binary.Write(buf, binary.LittleEndian, []byte(id.VertexName)); err != nil { + return nil, err + } + if err = binary.Write(buf, binary.LittleEndian, []byte(id.Offset)); err != nil { + return nil, err + } + return buf.Bytes(), nil +} + +// UnmarshalBinary decodes MessageID from the binary format +func (id *MessageID) UnmarshalBinary(data []byte) (err error) { + var r = bytes.NewReader(data) + var preamble = new(messageIDPreamble) + if err = binary.Read(r, binary.LittleEndian, preamble); err != nil { + return err + } + var vertexName = make([]byte, preamble.VertexNameLen) + if err = binary.Read(r, binary.LittleEndian, vertexName); err != nil { + return err + } + var offset = make([]byte, preamble.OffsetLen) + if err = binary.Read(r, binary.LittleEndian, offset); err != nil { + return err + } + id.VertexName = string(vertexName) + id.Offset = string(offset) + id.Index = preamble.Index + return nil +} + type headerPreamble struct { // message length MLen int32 @@ -73,10 +120,14 @@ func (h Header) MarshalBinary() (data []byte, err error) { if err != nil { return nil, err } + id, err := h.ID.MarshalBinary() + if err != nil { + return nil, err + } var preamble = headerPreamble{ MLen: int32(len(msgInfo)), MsgKind: h.Kind, - IDLen: int16(len(h.ID)), + IDLen: int16(len(id)), KeysLen: int16(len(h.Keys)), HeadersLen: int16(len(h.Headers)), } @@ -89,8 +140,11 @@ func (h Header) MarshalBinary() (data []byte, err error) { } else if n != int(preamble.MLen) { return nil, fmt.Errorf("expected to write msgInfo size of %d but got %d", preamble.MLen, n) } - if err = binary.Write(buf, binary.LittleEndian, []byte(h.ID)); err != nil { + n, err = buf.Write(id) + if err != nil { return nil, err + } else if n != int(preamble.IDLen) { + return nil, fmt.Errorf("expected to write id size of %d but got %d", preamble.IDLen, n) } for i := 0; i < len(h.Keys); i++ { if err = binary.Write(buf, binary.LittleEndian, int16(len(h.Keys[i]))); err != nil { @@ -136,8 +190,15 @@ func (h *Header) UnmarshalBinary(data []byte) (err error) { if err = msgInfo.UnmarshalBinary(msgInfoByte); err != nil { return err } - var id = make([]byte, preamble.IDLen) - if err = binary.Read(r, binary.LittleEndian, id); err != nil { + var idByte = make([]byte, preamble.IDLen) + n, err = r.Read(idByte) + if err != nil { + return err + } else if n != int(preamble.IDLen) { + return fmt.Errorf("expected to read id size of %d but got %d", preamble.IDLen, n) + } + var id = new(MessageID) + if err = id.UnmarshalBinary(idByte); err != nil { return err } keys := make([]string, 0) @@ -182,7 +243,7 @@ func (h *Header) UnmarshalBinary(data []byte) (err error) { } h.MessageInfo = *msgInfo h.Kind = preamble.MsgKind - h.ID = string(id) + h.ID = *id return err } diff --git a/pkg/isb/serde_test.go b/pkg/isb/serde_test.go index 5e1281d2a6..04da3862dc 100644 --- a/pkg/isb/serde_test.go +++ b/pkg/isb/serde_test.go @@ -91,7 +91,7 @@ func TestHeader(t *testing.T) { type fields struct { MessageInfo MessageInfo Kind MessageKind - ID string + ID MessageID Key []string Headers map[string]string } @@ -110,8 +110,12 @@ func TestHeader(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", - Key: []string{"TestKey", "TestKey2"}, + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, + Key: []string{"TestKey", "TestKey2"}, }, wantData: Header{ MessageInfo: MessageInfo{ @@ -119,7 +123,11 @@ func TestHeader(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey", "TestKey2"}, }, wantMarshalError: false, @@ -133,8 +141,12 @@ func TestHeader(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", - Key: []string{"TestKey", "TestKey2"}, + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, + Key: []string{"TestKey", "TestKey2"}, Headers: map[string]string{ "key1": "value1", "key2": "value2", @@ -146,7 +158,11 @@ func TestHeader(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey", "TestKey2"}, Headers: map[string]string{ "key1": "value1", @@ -260,7 +276,11 @@ func TestMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, }, Body: Body{ @@ -274,7 +294,11 @@ func TestMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, }, Body: Body{ @@ -300,7 +324,11 @@ func TestMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, Headers: map[string]string{ "key1": "value1", @@ -318,7 +346,11 @@ func TestMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, Headers: map[string]string{ "key1": "value1", @@ -381,7 +413,11 @@ func TestReadMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, Headers: map[string]string{ "key1": "value1", @@ -408,7 +444,11 @@ func TestReadMessage(t *testing.T) { IsLate: true, }, Kind: Data, - ID: "TestID", + ID: MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: 0, + }, Keys: []string{"TestKey"}, Headers: map[string]string{ "key1": "value1", diff --git a/pkg/isb/stores/jetstream/reader_test.go b/pkg/isb/stores/jetstream/reader_test.go index 1214d2d1e6..5227760a36 100644 --- a/pkg/isb/stores/jetstream/reader_test.go +++ b/pkg/isb/stores/jetstream/reader_test.go @@ -59,7 +59,7 @@ func TestJetStreamBufferRead(t *testing.T) { defer jw.Close() // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(20), startTime, nil, "testVertex") // Verify if buffer is full. for jw.isFull.Load() { select { diff --git a/pkg/isb/stores/jetstream/writer.go b/pkg/isb/stores/jetstream/writer.go index 8605b3290d..e3b7d1c979 100644 --- a/pkg/isb/stores/jetstream/writer.go +++ b/pkg/isb/stores/jetstream/writer.go @@ -201,7 +201,7 @@ func (jw *jetStreamWriter) asyncWrite(_ context.Context, messages []isb.Message, // nats.MsgId() is for exactly-once writing // we don't need to set MsgId for control message if message.Header.Kind != isb.WMB { - pubOpts = append(pubOpts, nats.MsgId(message.Header.ID)) + pubOpts = append(pubOpts, nats.MsgId(message.Header.ID.String())) } if future, err := jw.js.PublishMsgAsync(m, pubOpts...); err != nil { // nats.MsgId() is for exactly-once writing errs[index] = err @@ -279,7 +279,7 @@ func (jw *jetStreamWriter) syncWrite(_ context.Context, messages []isb.Message, // nats.MsgId() is for exactly-once writing // we don't need to set MsgId for control message if message.Header.Kind != isb.WMB { - pubOpts = append(pubOpts, nats.MsgId(message.Header.ID)) + pubOpts = append(pubOpts, nats.MsgId(message.Header.ID.String())) } if pubAck, err := jw.js.PublishMsg(m, pubOpts...); err != nil { errs[idx] = err @@ -293,7 +293,7 @@ func (jw *jetStreamWriter) syncWrite(_ context.Context, messages []isb.Message, } else { writeOffsets[idx] = &writeOffset{seq: pubAck.Sequence, partitionIdx: jw.partitionIdx} errs[idx] = nil - jw.log.Debugw("Succeeded to publish a message", zap.String("stream", pubAck.Stream), zap.Any("seq", pubAck.Sequence), zap.Bool("duplicate", pubAck.Duplicate), zap.String("msgID", message.Header.ID), zap.String("domain", pubAck.Domain)) + jw.log.Debugw("Succeeded to publish a message", zap.String("stream", pubAck.Stream), zap.Any("seq", pubAck.Sequence), zap.Bool("duplicate", pubAck.Duplicate), zap.Any("msgID", message.Header.ID), zap.String("domain", pubAck.Domain)) } } }(msg, index) diff --git a/pkg/isb/stores/jetstream/writer_test.go b/pkg/isb/stores/jetstream/writer_test.go index 7e57eb3c0b..51d9896615 100644 --- a/pkg/isb/stores/jetstream/writer_test.go +++ b/pkg/isb/stores/jetstream/writer_test.go @@ -44,11 +44,11 @@ func (f myForwardJetStreamTest) WhereTo(_ []string, _ []string, s string) ([]for } func (f myForwardJetStreamTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "test-vertex", message) } func (f myForwardJetStreamTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } // TestForwarderJetStreamBuffer is a test that is used to test forwarder with jetstream buffer @@ -95,7 +95,7 @@ func TestForwarderJetStreamBuffer(t *testing.T) { defer jw.Close() // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil, "testVertex") // Verify if buffer is not full. for jw.isFull.Load() { select { @@ -223,7 +223,7 @@ func TestJetStreamBufferWriterBufferFull(t *testing.T) { } // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil, "testVertex") // Add some data to buffer using write and verify no writes are performed when buffer is full _, errs := jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) @@ -239,7 +239,7 @@ func TestJetStreamBufferWriterBufferFull(t *testing.T) { time.Sleep(500 * time.Millisecond) } } - messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil) + messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil, "testVertex") _, errs = jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) for _, errMsg := range errs { @@ -280,7 +280,7 @@ func TestJetStreamBufferWriterBufferFull_DiscardLatest(t *testing.T) { } // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(2), startTime, nil, "testVertex") // Add some data to buffer using write and verify no writes are performed when buffer is full _, errs := jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) @@ -296,7 +296,7 @@ func TestJetStreamBufferWriterBufferFull_DiscardLatest(t *testing.T) { time.Sleep(500 * time.Millisecond) } } - messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil) + messages = testutils.BuildTestWriteMessages(int64(2), time.Unix(1636470001, 0), nil, "testVertex") _, errs = jw.Write(ctx, messages) assert.Equal(t, len(errs), 2) for _, errMsg := range errs { diff --git a/pkg/isb/stores/redis/read_test.go b/pkg/isb/stores/redis/read_test.go index 0294eff590..fbc4234788 100644 --- a/pkg/isb/stores/redis/read_test.go +++ b/pkg/isb/stores/redis/read_test.go @@ -68,7 +68,7 @@ func TestRedisQRead_Read(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(count, startTime, nil) + messages := testutils.BuildTestWriteMessages(count, startTime, nil, "testVertex") for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ Stream: rqr.GetStreamName(), @@ -100,7 +100,7 @@ func TestRedisCheckBacklog(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(count, startTime, nil) + messages := testutils.BuildTestWriteMessages(count, startTime, nil, "testVertex") for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ Stream: rqr.GetStreamName(), @@ -308,11 +308,11 @@ func (f forwardReadWritePerformance) WhereTo(_ []string, _ []string, _ string) ( } func (f forwardReadWritePerformance) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "testVertex", message) } func (f forwardReadWritePerformance) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "testVertex", writeMessageCh, message) } func (suite *ReadWritePerformance) SetupSuite() { @@ -392,7 +392,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatency() { suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) - writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil)...) + writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil, "testVertex")...) stopped := suite.isdf.Start() @@ -443,7 +443,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatencyPipelining() { suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) - writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil)...) + writeMessages = append(writeMessages, testutils.BuildTestWriteMessages(suite.count, testStartTime, nil, "testVertex")...) stopped := suite.isdf.Start() @@ -513,7 +513,7 @@ func generateLatencySlice(xMessages []redis.XMessage, suite *ReadWritePerformanc suite.NoError(err) } id, err := splitId(xMessage.ID) - offset, err := splitId(m.ID) + offset, err := splitId(m.ID.Offset) suite.NoError(err) // We store a difference of the id and the offset in the to stream. diff --git a/pkg/isb/stores/redis/write.go b/pkg/isb/stores/redis/write.go index 94876feffc..ce09507e9e 100644 --- a/pkg/isb/stores/redis/write.go +++ b/pkg/isb/stores/redis/write.go @@ -180,7 +180,7 @@ func (bw *BufferWrite) Write(_ context.Context, messages []isb.Message) ([]isb.O for idx, message := range messages { // Reference the Payload in Body directly when writing to Redis ISB to avoid extra marshaling. // TODO: revisit directly Payload reference when Body structure changes - errs[idx] = script.Run(ctx, bw.Client, []string{bw.GetHashKeyName(message.EventTime), bw.Stream}, message.Header.ID, message.Header, message.Body.Payload, bw.BufferWriteInfo.minId.String()).Err() + errs[idx] = script.Run(ctx, bw.Client, []string{bw.GetHashKeyName(message.EventTime), bw.Stream}, message.Header.ID.String(), message.Header, message.Body.Payload, bw.BufferWriteInfo.minId.String()).Err() } } else { var scriptMissing bool diff --git a/pkg/isb/stores/redis/write_test.go b/pkg/isb/stores/redis/write_test.go index 667bc7bd53..7341deb910 100644 --- a/pkg/isb/stores/redis/write_test.go +++ b/pkg/isb/stores/redis/write_test.go @@ -223,7 +223,7 @@ func buildTestWriteMessages(rqw *BufferWrite, count int64, startTime time.Time) var messages = make([]isb.Message, 0, count) var internalHashKeysMap map[string]bool var internalHashKeys = make([]string, 0) - messages = append(messages, testutils.BuildTestWriteMessages(count, startTime, nil)...) + messages = append(messages, testutils.BuildTestWriteMessages(count, startTime, nil, "testVertex")...) for i := int64(0); i < count; i++ { tmpTime := startTime.Add(time.Duration(i) * time.Minute) messages[i].EventTime = tmpTime @@ -239,7 +239,7 @@ func buildTestWriteMessages(rqw *BufferWrite, count int64, startTime time.Time) func TestLua(t *testing.T) { ctx := context.Background() client := redis.NewUniversalClient(redisOptions) - message := isb.Message{Header: isb.Header{ID: "0", MessageInfo: isb.MessageInfo{EventTime: testStartTime}}, Body: isb.Body{Payload: []byte("foo")}} + message := isb.Message{Header: isb.Header{ID: isb.MessageID{VertexName: "testVertex", Offset: "0"}, MessageInfo: isb.MessageInfo{EventTime: testStartTime}}, Body: isb.Body{Payload: []byte("foo")}} script := redis.NewScript(exactlyOnceInsertLuaScript) var hashName = "{step-1}:1234567890:hash-foo" @@ -352,11 +352,11 @@ func (f myForwardRedisTest) WhereTo(_ []string, _ []string, _ string) ([]forward } func (f myForwardRedisTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (f myForwardRedisTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } // TestNewInterStepDataForwardRedis is used to read data from one step to another using redis as the Inter-Step Buffer @@ -487,7 +487,7 @@ func TestXTrimOnIsFull(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil, "testVertex") // Add 10 messages for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ @@ -552,7 +552,7 @@ func TestSetWriteInfo(t *testing.T) { // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil, "testVertex") // Add 10 messages for _, msg := range messages { err := client.Client.XAdd(ctx, &redis.XAddArgs{ diff --git a/pkg/isb/stores/simplebuffer/buffer_test.go b/pkg/isb/stores/simplebuffer/buffer_test.go index 20cf292d21..427a7a0168 100644 --- a/pkg/isb/stores/simplebuffer/buffer_test.go +++ b/pkg/isb/stores/simplebuffer/buffer_test.go @@ -38,7 +38,7 @@ func TestNewSimpleBuffer(t *testing.T) { assert.Equal(t, sb.IsEmpty(), true) startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil, "testVertex") sb.Write(ctx, writeMessages[0:5]) assert.Equal(t, int64(5), sb.writeIdx) assert.Equal(t, int64(0), sb.readIdx) @@ -88,7 +88,7 @@ func TestNewSimpleBuffer_BufferFullWritingStrategyIsDiscard(t *testing.T) { assert.Equal(t, sb.IsEmpty(), true) startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(count, startTime, nil, "testVertex") // try to write 3 messages, it should fail (we have only space for 2) // the first 2 messages should be written, the last one should be discarded and returns us a NoRetryableError. diff --git a/pkg/isb/testutils/rw.go b/pkg/isb/testutils/rw.go index c15250049a..84805fc5d2 100644 --- a/pkg/isb/testutils/rw.go +++ b/pkg/isb/testutils/rw.go @@ -38,7 +38,7 @@ type PayloadForTest struct { } // BuildTestWriteMessages builds test isb.Message which can be used for testing. -func BuildTestWriteMessages(count int64, startTime time.Time, keys []string) []isb.Message { +func BuildTestWriteMessages(count int64, startTime time.Time, keys []string, vertexName string) []isb.Message { var messages = make([]isb.Message, 0, count) for i := int64(0); i < count; i++ { tmpTime := startTime.Add(time.Duration(i) * time.Second) @@ -52,7 +52,11 @@ func BuildTestWriteMessages(count int64, startTime time.Time, keys []string) []i MessageInfo: isb.MessageInfo{ EventTime: tmpTime, }, - ID: fmt.Sprintf("%d-testVertex-0-0", i), // TODO: hard coded ID suffix ATM, make configurable if needed + ID: isb.MessageID{ + VertexName: vertexName, + Offset: "0-0", + Index: int32(i), + }, // TODO: hard coded ID suffix ATM, make configurable if needed Keys: keys, Headers: map[string]string{ "key1": "value1", @@ -82,7 +86,7 @@ func BuildTestWindowRequests(count int64, startTime time.Time, windowOp window.O // BuildTestReadMessages builds test isb.ReadMessage which can be used for testing. func BuildTestReadMessages(count int64, startTime time.Time, keys []string) []isb.ReadMessage { - writeMessages := BuildTestWriteMessages(count, startTime, keys) + writeMessages := BuildTestWriteMessages(count, startTime, keys, "testVertex") var readMessages = make([]isb.ReadMessage, count) for idx, writeMessage := range writeMessages { @@ -97,12 +101,12 @@ func BuildTestReadMessages(count int64, startTime time.Time, keys []string) []is // BuildTestReadMessagesIntOffset builds test isb.ReadMessage which can be used for testing. func BuildTestReadMessagesIntOffset(count int64, startTime time.Time, keys []string) []isb.ReadMessage { - writeMessages := BuildTestWriteMessages(count, startTime, keys) + writeMessages := BuildTestWriteMessages(count, startTime, keys, "testVertex") var readMessages = make([]isb.ReadMessage, count) for idx, writeMessage := range writeMessages { - splitStr := strings.Split(writeMessage.Header.ID, "-") - offset, _ := strconv.Atoi(splitStr[0]) + splitStr := strings.Split(writeMessage.Header.ID.String(), "-") + offset, _ := strconv.Atoi(splitStr[len(splitStr)-1]) readMessages[idx] = isb.ReadMessage{ Message: writeMessage, ReadOffset: isb.NewSimpleIntPartitionOffset(int64(offset), 0), diff --git a/pkg/isb/testutils/udf.go b/pkg/isb/testutils/udf.go index 5aecc363b5..47c6a8a61c 100644 --- a/pkg/isb/testutils/udf.go +++ b/pkg/isb/testutils/udf.go @@ -23,7 +23,7 @@ import ( ) // CopyUDFTestApply applies a copy UDF that simply copies the input to output. -func CopyUDFTestApply(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { +func CopyUDFTestApply(ctx context.Context, vertexName string, readMessage *isb.ReadMessage) ([]*isb.WriteMessage, error) { _ = ctx offset := readMessage.ReadOffset payload := readMessage.Body.Payload @@ -38,8 +38,11 @@ func CopyUDFTestApply(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb writeMessage := isb.Message{ Header: isb.Header{ MessageInfo: parentPaneInfo, - ID: offset.String(), - Keys: keys, + ID: isb.MessageID{ + VertexName: vertexName, + Offset: offset.String(), + }, + Keys: keys, }, Body: isb.Body{ Payload: result, @@ -48,7 +51,7 @@ func CopyUDFTestApply(ctx context.Context, readMessage *isb.ReadMessage) ([]*isb return []*isb.WriteMessage{{Message: writeMessage}}, nil } -func CopyUDFTestApplyStream(ctx context.Context, readMessage *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { +func CopyUDFTestApplyStream(ctx context.Context, vertexName string, writeMessageCh chan<- isb.WriteMessage, readMessage *isb.ReadMessage) error { defer close(writeMessageCh) _ = ctx offset := readMessage.ReadOffset @@ -64,7 +67,7 @@ func CopyUDFTestApplyStream(ctx context.Context, readMessage *isb.ReadMessage, w writeMessage := isb.Message{ Header: isb.Header{ MessageInfo: parentPaneInfo, - ID: offset.String(), + ID: isb.MessageID{VertexName: vertexName, Offset: offset.String()}, Keys: keys, }, Body: isb.Body{ diff --git a/pkg/isbsvc/redis_service_test.go b/pkg/isbsvc/redis_service_test.go index 444db9eb92..4a4beff62a 100644 --- a/pkg/isbsvc/redis_service_test.go +++ b/pkg/isbsvc/redis_service_test.go @@ -51,7 +51,7 @@ func TestIsbsRedisSvc_Buffers(t *testing.T) { // Verify // Add some data startTime := time.Unix(1636470000, 0) - messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil) + messages := testutils.BuildTestWriteMessages(int64(10), startTime, nil, "testVertex") // Add 10 messages for _, msg := range messages { err := redisClient.Client.XAdd(ctx, &goredis.XAddArgs{ diff --git a/pkg/reduce/data_forward_test.go b/pkg/reduce/data_forward_test.go index 4d1c9e4bbe..a617311d30 100644 --- a/pkg/reduce/data_forward_test.go +++ b/pkg/reduce/data_forward_test.go @@ -156,7 +156,10 @@ func (f CounterReduceTest) ApplyReduce(_ context.Context, partitionID *partition MessageInfo: isb.MessageInfo{ EventTime: partitionID.End.Add(-1 * time.Millisecond), }, - ID: "msgID", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"result"}, }, Body: isb.Body{Payload: b}, @@ -213,7 +216,10 @@ func (s SessionSumReduceTest) ApplyReduce(ctx context.Context, partitionID *part MessageInfo: isb.MessageInfo{ EventTime: partitionID.End.Add(-1 * time.Millisecond), }, - ID: "msgID", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{win.Keys()[0]}, }, Body: isb.Body{Payload: b}, @@ -275,7 +281,10 @@ func (s SumReduceTest) ApplyReduce(_ context.Context, partitionID *partition.ID, MessageInfo: isb.MessageInfo{ EventTime: partitionID.End.Add(-1 * time.Millisecond), }, - ID: "msgID", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{k}, }, Body: isb.Body{Payload: b}, @@ -333,7 +342,10 @@ func (m MaxReduceTest) ApplyReduce(_ context.Context, partitionID *partition.ID, MessageInfo: isb.MessageInfo{ EventTime: partitionID.End.Add(-1 * time.Millisecond), }, - ID: "msgID", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{k}, }, Body: isb.Body{Payload: b}, @@ -620,7 +632,7 @@ func TestReduceDataForward_IdleWM(t *testing.T) { assert.Equal(t, isb.Header{ MessageInfo: isb.MessageInfo{}, Kind: 0, - ID: "", + ID: isb.MessageID{}, }, msgs[i].Header) } @@ -648,7 +660,10 @@ func TestReduceDataForward_IdleWM(t *testing.T) { } assert.Equal(t, isb.Data, msgs[1].Kind) // in the test ApplyUDF above we've set the final message to have ID="msgID" - assert.Equal(t, "msgID", msgs[1].ID) + assert.Equal(t, isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, msgs[1].ID) // in the test ApplyUDF above we've set the final message to have eventTime = partitionID.End-1ms assert.Equal(t, int64(1679961604999), msgs[1].EventTime.UnixMilli()) var result PayloadForTest @@ -663,7 +678,7 @@ func TestReduceDataForward_IdleWM(t *testing.T) { assert.Equal(t, isb.Header{ MessageInfo: isb.MessageInfo{}, Kind: 0, - ID: "", + ID: isb.MessageID{}, }, msgs[i].Header) } @@ -1628,7 +1643,11 @@ func buildMessagesForReduce(count int, key string, publishTime time.Time) []isb. MessageInfo: isb.MessageInfo{ EventTime: publishTime, }, - ID: fmt.Sprintf("%d", i), + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + Index: int32(i), + }, Keys: []string{key}, }, Body: isb.Body{Payload: result}, @@ -1764,7 +1783,10 @@ func buildIsbMessage(messageValue int, eventTime time.Time) isb.Message { MessageInfo: isb.MessageInfo{ EventTime: eventTime, }, - ID: fmt.Sprintf("%d", messageValue), + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: messageKey, }, Body: isb.Body{Payload: result}, @@ -1851,7 +1873,10 @@ func buildIsbMessageAllowedLatency(messageValue int, eventTime time.Time) isb.Me EventTime: eventTime, IsLate: true, }, - ID: fmt.Sprintf("%d", messageValue), + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: messageKey, }, Body: isb.Body{Payload: result}, diff --git a/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go b/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go index 02657abe31..5b8a1e3a7a 100644 --- a/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go +++ b/pkg/reduce/pbq/wal/aligned/fs/wal_segment_test.go @@ -280,7 +280,7 @@ func Test_batchSyncWithMaxBatchSize(t *testing.T) { assert.NoError(t, err) err = wal.Write(&message) assert.NoError(t, err) - assert.Equal(t, int64(308), tempWAL.prevSyncedWOffset) + assert.Equal(t, int64(322), tempWAL.prevSyncedWOffset) err = wal.Close() assert.NoError(t, err) @@ -364,7 +364,7 @@ func Test_batchSyncWithSyncDuration(t *testing.T) { message := writeMessages[0] storePrevSyncedTime := tempWAL.prevSyncedTime err = wal.Write(&message) - assert.Equal(t, int64(173), tempWAL.prevSyncedWOffset) + assert.Equal(t, int64(180), tempWAL.prevSyncedWOffset) assert.NotEqual(t, storePrevSyncedTime, tempWAL.prevSyncedTime) assert.NoError(t, err) diff --git a/pkg/reduce/pnf/pnf.go b/pkg/reduce/pnf/pnf.go index 1523587fc2..b1fffb1494 100644 --- a/pkg/reduce/pnf/pnf.go +++ b/pkg/reduce/pnf/pnf.go @@ -350,7 +350,7 @@ func (pf *ProcessAndForward) whereToStep(writeMessages []*isb.WriteMessage) map[ var to []forwarder.VertexBuffer var err error for _, msg := range writeMessages { - to, err = pf.whereToDecider.WhereTo(msg.Keys, msg.Tags, msg.ID) + to, err = pf.whereToDecider.WhereTo(msg.Keys, msg.Tags, msg.ID.String()) if err != nil { metrics.PlatformError.With(map[string]string{ metrics.LabelVertex: pf.vertexName, diff --git a/pkg/reduce/pnf/pnf_test.go b/pkg/reduce/pnf/pnf_test.go index 304af5d01d..a214ba8fe9 100644 --- a/pkg/reduce/pnf/pnf_test.go +++ b/pkg/reduce/pnf/pnf_test.go @@ -54,7 +54,7 @@ func (f *forwardTest) WhereTo(_ []string, _ []string, s string) ([]forwarder.Ver // TestWriteToBuffer tests two BufferFullWritingStrategies: 1. discarding the latest message and 2. retrying writing until context is cancelled. func TestWriteToBuffer(t *testing.T) { testStartTime := time.Unix(1636470000, 0).UTC() - windowResponse := testutils.BuildTestWriteMessages(int64(15), testStartTime, nil) + windowResponse := testutils.BuildTestWriteMessages(int64(15), testStartTime, nil, "testVertex") tests := []struct { name string diff --git a/pkg/shared/callback/callback.go b/pkg/shared/callback/callback.go new file mode 100644 index 0000000000..3f8b6fdaaf --- /dev/null +++ b/pkg/shared/callback/callback.go @@ -0,0 +1,285 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package callback + +import ( + "bytes" + "context" + "crypto/tls" + "encoding/json" + "errors" + "fmt" + "net/http" + "time" + + lru "github.com/hashicorp/golang-lru/v2" + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" +) + +// Uploader uploads the callback messages to the callback endpoint. +type Uploader struct { + vertexName string + pipelineName string + clientsCache *lru.Cache[string, *http.Client] + opts *Options +} + +// NewUploader creates a new callback Uploader. +func NewUploader(ctx context.Context, vertexName string, pipelineName string, opts ...OptionFunc) *Uploader { + dOpts := DefaultOptions(ctx) + for _, opt := range opts { + opt(dOpts) + } + + clientCache, _ := lru.NewWithEvict[string, *http.Client](dOpts.cacheSize, func(key string, value *http.Client) { + // Close the client when it's removed from the cache + value.CloseIdleConnections() + }) + + return &Uploader{ + vertexName: vertexName, + pipelineName: pipelineName, + clientsCache: clientCache, + opts: dOpts, + } +} + +// Request is the struct that holds the data to be sent in the POST request +type Request struct { + // Vertex is the name of the vertex + Vertex string `json:"vertex"` + // Pipeline is the name of the pipeline + Pipeline string `json:"pipeline"` + // UUID is the unique identifier of the message + ID string `json:"id"` + // CbTime is the time when the callback was made + CbTime int64 `json:"cb_time"` + // Tags is the list of tags associated with the message + Tags []string `json:"tags,omitempty"` // Add 'omitempty' here + // FromVertex is the name of the vertex from which the message was sent + FromVertex string `json:"from_vertex"` +} + +// NonSinkVertexCallback groups callbacks based on their callback URL and makes a POST request for each callback group +// for non-sink vertices. If the callbackHeaderKey is set, it writes all the callback requests to the callbackURL. +// In case of failure while writing the url from the headers, it writes all the callback requests to the callbackURL. +// The callback URL could change because of the affinity towards colocating the callbacks to the same endpoint server +// that initiated the request (each request could be from a different server in the pool). +func (u *Uploader) NonSinkVertexCallback(ctx context.Context, messagePairs []isb.ReadWriteMessagePair) error { + // Create a map to hold groups of messagePairs for each callback URL + callbackUrlMap := make(map[string][]Request) + + // Iterate through each pair + for _, pair := range messagePairs { + + for _, msg := range pair.WriteMessages { + + // Extract Callback URL from message headers or use the default callback URL + var callbackURL string + if cbURL, ok := msg.Headers[u.opts.callbackHeaderKey]; !ok { + if u.opts.callbackURL == "" { + return fmt.Errorf("callback URL not found in headers and default callback URL is not set") + } + callbackURL = u.opts.callbackURL + } else { + callbackURL = cbURL + } + + // Extract UUID from pair headers + uuid, ok := msg.Headers[dfv1.KeyMetaID] + if !ok { + // Return an error if UUID is not found in pair headers + return errors.New("ID not found in message headers") + } + + // Create a new CallbackResponse + callbackTime := time.Now().UnixMilli() + newObject := Request{ + Vertex: u.vertexName, + Pipeline: u.pipelineName, + ID: uuid, + CbTime: callbackTime, + Tags: msg.Tags, + // the read message id has the vertex name of the vertex that sent the message + FromVertex: pair.ReadMessage.ID.VertexName, + } + // if the callback URL is not present in the map, create a new slice + if _, ok = callbackUrlMap[callbackURL]; !ok { + callbackUrlMap[callbackURL] = make([]Request, 0) + } + + // append the new callback response to the map grouped by the callback URL + callbackUrlMap[callbackURL] = append(callbackUrlMap[callbackURL], newObject) + } + } + + if err := u.executeCallback(ctx, callbackUrlMap); err != nil { + return fmt.Errorf("error executing callback: %w", err) + } + + return nil +} + +// SinkVertexCallback groups callbacks based on their callback URL present in the headers and sends a POST request for +// each callback group for sink vertices. If the callback header is not set, it writes all the callback requests to the callbackURL. +// In case of failure while writing the url from the headers, it writes all the callback requests to the callbackURL. +func (u *Uploader) SinkVertexCallback(ctx context.Context, messages []isb.Message) error { + // Create a map to hold set of callback requests for each callback URL + callbackUrlMap := make(map[string][]Request) + + for _, msg := range messages { + // Extract Callback URL from message headers or use the default callback URL + var callbackURL string + if cbURL, ok := msg.Headers[u.opts.callbackHeaderKey]; !ok { + if u.opts.callbackURL == "" { + return fmt.Errorf("callback URL not found in headers and default callback URL is not set") + } + callbackURL = u.opts.callbackURL + } else { + callbackURL = cbURL + } + + // Extract UUID from message headers + uuid, ok := msg.Headers[dfv1.KeyMetaID] + if !ok { + // Return an error if UUID is not found in pair headers + return errors.New("ID not found in message headers") + } + + // Create a new CallbackResponse + // for sink vertex since the message is not paired(terminal vertex). The from_vertex can be + // extracted from the message itself unlike the non-sink vertex where it is extracted from the + // read message. + callbackTime := time.Now().UnixMilli() + newObject := Request{ + Vertex: u.vertexName, + Pipeline: u.pipelineName, + ID: uuid, + CbTime: callbackTime, + FromVertex: msg.ID.VertexName, + } + + // if the callback URL is not present in the map, create a new slice + if _, ok = callbackUrlMap[callbackURL]; !ok { + callbackUrlMap[callbackURL] = make([]Request, 0) + } + + // Add new CallbackResponse to map, grouped by the Callback URL + callbackUrlMap[callbackURL] = append(callbackUrlMap[callbackURL], newObject) + } + + if err := u.executeCallback(ctx, callbackUrlMap); err != nil { + return fmt.Errorf("error executing callback: %w", err) + } + + return nil +} + +// executeCallback sends POST requests to the provided callback URLs with the corresponding request bodies. +// It returns an error if any of the requests fail. +func (u *Uploader) executeCallback(ctx context.Context, callbackUrlMap map[string][]Request) error { + var failedRequests []Request + + for url, requests := range callbackUrlMap { + err := u.sendRequest(ctx, url, requests) + if err != nil { + u.opts.logger.Errorw("Failed to send request, will try writing to the callback URL", + zap.String("url", url), + zap.Error(err), + ) + failedRequests = append(failedRequests, requests...) + } + } + + if len(failedRequests) > 0 && u.opts.callbackURL != "" { + err := u.sendRequest(ctx, u.opts.callbackURL, failedRequests) + if err != nil { + u.opts.logger.Errorw("Failed to send request to callback URL, skipping the callback requests", + zap.String("url", u.opts.callbackURL), + zap.Error(err), + ) + } + } + + return nil +} + +// sendRequest sends a POST request to the provided URL with the provided requests. +// It returns an error if the request fails or if the response status is not OK. +func (u *Uploader) sendRequest(ctx context.Context, url string, requests []Request) error { + // get a cached client + client := u.GetClient(url) + + body, err := json.Marshal(requests) + if err != nil { + return fmt.Errorf("failed to marshal requests: %w", err) + } + + req, err := http.NewRequestWithContext(ctx, http.MethodPost, url, bytes.NewBuffer(body)) + if err != nil { + return fmt.Errorf("failed to create request: %w", err) + } + + req.Header.Set("Content-Type", "application/json") + + resp, err := client.Do(req) + if err != nil { + return fmt.Errorf("failed to send request: %w", err) + } + + defer resp.Body.Close() + + if resp.StatusCode > 299 { + return fmt.Errorf("received non-OK response status: %s", resp.Status) + } + + return nil +} + +// GetClient returns a client for the given URL from the cache +// If the client is not in the cache, a new one is created. +func (u *Uploader) GetClient(url string) *http.Client { + // Check if the client is in the cache + if client, ok := u.clientsCache.Get(url); ok { + return client + } + + // If the client is not in the cache, create a new one + client := &http.Client{ + Timeout: u.opts.httpTimeout, + Transport: &http.Transport{ + // we don't need authentication for the callback, since we expect it to be a local service. + TLSClientConfig: &tls.Config{ + InsecureSkipVerify: true, + }, + }, + } + + // Add the new client to the cache + u.clientsCache.Add(url, client) + + return client +} + +// Close closes all clients in the cache +func (u *Uploader) Close() { + // clear the cache, which will call the onEvicted method for each client + u.clientsCache.Purge() +} diff --git a/pkg/shared/callback/callback_test.go b/pkg/shared/callback/callback_test.go new file mode 100644 index 0000000000..c928e878d7 --- /dev/null +++ b/pkg/shared/callback/callback_test.go @@ -0,0 +1,317 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package callback + +import ( + "context" + "encoding/json" + "io" + "net/http" + "net/http/httptest" + "testing" + + "github.com/stretchr/testify/assert" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" +) + +func TestNonSinkVertexCallback(t *testing.T) { + expected := []map[string]interface{}{ + { + "vertex": "testVertex", + "pipeline": "testPipeline", + "id": "XXXX", + "from_vertex": "from-vertex", + }, + } + + server := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) { + // get request body and convert it to the text + bodyBytes, _ := io.ReadAll(req.Body) + var actual []map[string]interface{} + err := json.Unmarshal(bodyBytes, &actual) + assert.Nil(t, err) + + // remove the cb_time field + for _, item := range actual { + delete(item, "cb_time") + } + + // compare the expected and actual body + assert.Equal(t, expected, actual) + + // send 204 response + rw.WriteHeader(http.StatusNoContent) + })) + + defer server.Close() + + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline", WithCallbackURL(server.URL)) + + messagePairs := []isb.ReadWriteMessagePair{ + { + ReadMessage: &isb.ReadMessage{ + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaCallbackURL: server.URL, + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + }, + WriteMessages: []*isb.WriteMessage{ + { + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaCallbackURL: server.URL, + dfv1.KeyMetaID: "XXXX", + }, + }, + }, + }, + }, + }, + } + + err := cp.NonSinkVertexCallback(ctx, messagePairs) + assert.Nil(t, err) +} + +func TestNonSinkVertexCallback_NoCallbackURL(t *testing.T) { + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline") + + messagePairs := []isb.ReadWriteMessagePair{ + { + ReadMessage: &isb.ReadMessage{ + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + }, + WriteMessages: []*isb.WriteMessage{ + { + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + }, + }, + }, + }, + }, + } + + err := cp.NonSinkVertexCallback(ctx, messagePairs) + assert.NotNil(t, err) + assert.Contains(t, err.Error(), "callback URL not found in headers and default callback URL is not set") +} + +func TestNonSinkVertexCallback_CallbackURLSet(t *testing.T) { + expected := []map[string]interface{}{ + { + "vertex": "testVertex", + "pipeline": "testPipeline", + "id": "XXXX", + "from_vertex": "from-vertex", + }, + } + + server := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) { + bodyBytes, _ := io.ReadAll(req.Body) + var actual []map[string]interface{} + err := json.Unmarshal(bodyBytes, &actual) + assert.Nil(t, err) + + for _, item := range actual { + delete(item, "cb_time") + } + + assert.Equal(t, expected, actual) + rw.WriteHeader(http.StatusNoContent) + })) + + defer server.Close() + + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline", WithCallbackURL(server.URL)) + + messagePairs := []isb.ReadWriteMessagePair{ + { + ReadMessage: &isb.ReadMessage{ + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + }, + WriteMessages: []*isb.WriteMessage{ + { + Message: isb.Message{ + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + }, + }, + }, + }, + }, + } + + err := cp.NonSinkVertexCallback(ctx, messagePairs) + assert.Nil(t, err) +} + +func TestSinkVertexCallback(t *testing.T) { + expected := []map[string]interface{}{ + { + "vertex": "testVertex", + "pipeline": "testPipeline", + "id": "XXXX", + "from_vertex": "from-vertex", + }, + } + + server := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) { + // get request body and convert it to the text + bodyBytes, _ := io.ReadAll(req.Body) + var actual []map[string]interface{} + err := json.Unmarshal(bodyBytes, &actual) + assert.Nil(t, err) + + // remove the cb_time field + for _, item := range actual { + delete(item, "cb_time") + } + + // compare the expected and actual body + assert.Equal(t, expected, actual) + + // send 204 response + rw.WriteHeader(http.StatusNoContent) + })) + + defer server.Close() + + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline", WithCallbackURL(server.URL)) + + messages := []isb.Message{ + { + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaCallbackURL: server.URL, + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + } + + err := cp.SinkVertexCallback(ctx, messages) + assert.Nil(t, err) +} + +func TestSinkVertexCallback_CallbackURLSet(t *testing.T) { + expected := []map[string]interface{}{ + { + "vertex": "testVertex", + "pipeline": "testPipeline", + "id": "XXXX", + "from_vertex": "from-vertex", + }, + } + + server := httptest.NewServer(http.HandlerFunc(func(rw http.ResponseWriter, req *http.Request) { + bodyBytes, _ := io.ReadAll(req.Body) + var actual []map[string]interface{} + err := json.Unmarshal(bodyBytes, &actual) + assert.Nil(t, err) + + for _, item := range actual { + delete(item, "cb_time") + } + + assert.Equal(t, expected, actual) + rw.WriteHeader(http.StatusNoContent) + })) + + defer server.Close() + + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline", WithCallbackURL(server.URL)) + + messages := []isb.Message{ + { + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + } + + err := cp.SinkVertexCallback(ctx, messages) + assert.Nil(t, err) +} + +func TestSinkVertexCallback_NoCallbackURL(t *testing.T) { + ctx := context.Background() + cp := NewUploader(ctx, "testVertex", "testPipeline") + + messages := []isb.Message{ + { + Header: isb.Header{ + Headers: map[string]string{ + dfv1.KeyMetaID: "XXXX", + }, + ID: isb.MessageID{ + VertexName: "from-vertex", + }, + }, + }, + } + + err := cp.SinkVertexCallback(ctx, messages) + assert.NotNil(t, err) + assert.Contains(t, err.Error(), "callback URL not found in headers and default callback URL is not set") +} diff --git a/pkg/shared/callback/options.go b/pkg/shared/callback/options.go new file mode 100644 index 0000000000..ccf47cdc9a --- /dev/null +++ b/pkg/shared/callback/options.go @@ -0,0 +1,83 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package callback + +import ( + "context" + "time" + + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/shared/logging" +) + +// Options holds optional parameters for the Uploader. +type Options struct { + // httpTimeout specifies the timeout for HTTP requests. + httpTimeout time.Duration + // cacheSize specifies the size of the LRU cache for HTTP clients. + cacheSize int + // callbackHeaderKey specifies the key using which the callback URL + // is passed in the headers. + callbackHeaderKey string + // callbackURL specifies the URL to which the callback is sent. + callbackURL string + // logger is the logger for the publisher. + logger *zap.SugaredLogger +} + +// DefaultOptions returns the default options. +func DefaultOptions(ctx context.Context) *Options { + return &Options{ + httpTimeout: 10 * time.Second, + cacheSize: 50, + logger: logging.FromContext(ctx), + callbackHeaderKey: dfv1.KeyMetaCallbackURL, + } +} + +// OptionFunc is a function that applies an option to the Uploader. +type OptionFunc func(*Options) + +// WithHTTPTimeout sets the HTTP timeout. +func WithHTTPTimeout(timeout time.Duration) OptionFunc { + return func(o *Options) { + o.httpTimeout = timeout + } +} + +// WithLRUCacheSize sets the LRU cache size. +func WithLRUCacheSize(size int) OptionFunc { + return func(o *Options) { + o.cacheSize = size + } +} + +// WithCallbackURL sets the callback URL. +func WithCallbackURL(url string) OptionFunc { + return func(o *Options) { + o.callbackURL = url + } +} + +// WithLogger sets the logger. +func WithLogger(logger *zap.SugaredLogger) OptionFunc { + return func(o *Options) { + o.logger = logger + } +} diff --git a/pkg/shared/callback/options_test.go b/pkg/shared/callback/options_test.go new file mode 100644 index 0000000000..89188e27f7 --- /dev/null +++ b/pkg/shared/callback/options_test.go @@ -0,0 +1,50 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package callback + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "go.uber.org/zap" +) + +func TestOptions(t *testing.T) { + ctx := context.Background() + + // Create default options + opts := DefaultOptions(ctx) + + // Check default values + assert.Equal(t, 10*time.Second, opts.httpTimeout) + assert.Equal(t, 50, opts.cacheSize) + assert.NotNil(t, opts.logger) + + // Modify options + WithHTTPTimeout(20 * time.Second)(opts) + WithLRUCacheSize(100)(opts) + WithCallbackURL("http://example.com")(opts) + WithLogger(zap.NewNop().Sugar())(opts) + + // Check modified values + assert.Equal(t, 20*time.Second, opts.httpTimeout) + assert.Equal(t, 100, opts.cacheSize) + assert.Equal(t, "http://example.com", opts.callbackURL) + assert.IsType(t, &zap.SugaredLogger{}, opts.logger) +} diff --git a/pkg/shuffle/shuffle_test.go b/pkg/shuffle/shuffle_test.go index 1461fb377a..9a3b68366c 100644 --- a/pkg/shuffle/shuffle_test.go +++ b/pkg/shuffle/shuffle_test.go @@ -120,7 +120,7 @@ func isSameShuffleDistribution(a, b map[int32][]*isb.Message) bool { func buildTestMessagesWithDistinctKeys(size int64) []*isb.Message { // build test messages - messages := testutils.BuildTestWriteMessages(size, time.Now(), nil) + messages := testutils.BuildTestWriteMessages(size, time.Now(), nil, "testVertex") // set keys for test messages var res []*isb.Message for index := 0; index < len(messages); index++ { diff --git a/pkg/sinks/blackhole/blackhole_test.go b/pkg/sinks/blackhole/blackhole_test.go index 464a8a568b..95516f2499 100644 --- a/pkg/sinks/blackhole/blackhole_test.go +++ b/pkg/sinks/blackhole/blackhole_test.go @@ -32,7 +32,7 @@ func TestBlackhole_Start(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ AbstractVertex: dfv1.AbstractVertex{ diff --git a/pkg/sinks/forward/forward.go b/pkg/sinks/forward/forward.go index ecb0f7c133..7522b7135d 100644 --- a/pkg/sinks/forward/forward.go +++ b/pkg/sinks/forward/forward.go @@ -278,6 +278,11 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { } metrics.AckMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + if df.opts.cbPublisher != nil { + if err = df.opts.cbPublisher.SinkVertexCallback(ctx, writeMessages); err != nil { + df.opts.logger.Error("Failed to execute callback", zap.Error(err)) + } + } // ProcessingTimes of the entire forwardAChunk metrics.ForwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } diff --git a/pkg/sinks/forward/forward_test.go b/pkg/sinks/forward/forward_test.go index e169fb11ff..62750b0c9a 100644 --- a/pkg/sinks/forward/forward_test.go +++ b/pkg/sinks/forward/forward_test.go @@ -155,7 +155,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) fetchWatermark := &testForwardFetcher{} @@ -173,7 +173,8 @@ func TestNewDataForward(t *testing.T) { testReadBatchSize := int(batchSize / 2) msgs := to1.GetMessages(testReadBatchSize) - for ; msgs[testReadBatchSize-1].ID == ""; msgs = to1.GetMessages(testReadBatchSize) { + emptyId := isb.MessageID{} + for ; msgs[testReadBatchSize-1].ID == emptyId; msgs = to1.GetMessages(testReadBatchSize) { select { case <-ctx.Done(): if ctx.Err() == context.DeadlineExceeded { @@ -337,7 +338,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep []isb.Message - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil, "testVertex") messageToStep = append(messageToStep, writeMessages[0:value.batchSize+1]...) _, _, err = f.writeToSink(ctx, buffer, messageToStep, false) diff --git a/pkg/sinks/forward/options.go b/pkg/sinks/forward/options.go index 426b398e88..f18dbe2620 100644 --- a/pkg/sinks/forward/options.go +++ b/pkg/sinks/forward/options.go @@ -22,6 +22,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/shared/callback" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/sinks/sinker" ) @@ -38,6 +39,8 @@ type options struct { fbSinkWriter sinker.SinkWriter // logger is used to pass the logger variable logger *zap.SugaredLogger + // cbPublisher is the callback publisher for the vertex. + cbPublisher *callback.Uploader } type Option func(*options) error @@ -90,3 +93,11 @@ func WithFbSinkWriter(sinkWriter sinker.SinkWriter) Option { return nil } } + +// WithCallbackUploader sets the callback uploader for the vertex +func WithCallbackUploader(cp *callback.Uploader) Option { + return func(o *options) error { + o.cbPublisher = cp + return nil + } +} diff --git a/pkg/sinks/forward/shutdown_test.go b/pkg/sinks/forward/shutdown_test.go index bf63fb945b..4dc004280d 100644 --- a/pkg/sinks/forward/shutdown_test.go +++ b/pkg/sinks/forward/shutdown_test.go @@ -57,7 +57,7 @@ func TestShutDown(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -97,7 +97,7 @@ func TestShutDown(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/sinks/logger/log.go b/pkg/sinks/logger/log.go index 3d7e31f9ac..776048b05e 100644 --- a/pkg/sinks/logger/log.go +++ b/pkg/sinks/logger/log.go @@ -72,7 +72,7 @@ func (t *ToLog) Write(_ context.Context, messages []isb.Message) ([]isb.Offset, hStr.WriteString(fmt.Sprintf("%s: %s, ", k, v)) } logSinkWriteCount.With(map[string]string{metrics.LabelVertex: t.name, metrics.LabelPipeline: t.pipelineName}).Inc() - log.Println(prefix, " Payload - ", string(message.Payload), " Keys - ", message.Keys, " EventTime - ", message.EventTime.UnixMilli(), " Headers - ", hStr.String()) + log.Println(prefix, " Payload - ", string(message.Payload), " Keys - ", message.Keys, " EventTime - ", message.EventTime.UnixMilli(), " Headers - ", hStr.String(), " ID - ", message.ID.String()) } return nil, make([]error, len(messages)) } diff --git a/pkg/sinks/logger/log_test.go b/pkg/sinks/logger/log_test.go index 68bfcaa053..322e4e338f 100644 --- a/pkg/sinks/logger/log_test.go +++ b/pkg/sinks/logger/log_test.go @@ -32,7 +32,7 @@ func TestToLog_Start(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ AbstractVertex: dfv1.AbstractVertex{ diff --git a/pkg/sinks/sink.go b/pkg/sinks/sink.go index da4901463e..f18f545127 100644 --- a/pkg/sinks/sink.go +++ b/pkg/sinks/sink.go @@ -19,6 +19,7 @@ package sinks import ( "context" "fmt" + "os" "sync" "go.uber.org/zap" @@ -33,6 +34,7 @@ import ( "github.com/numaproj/numaflow/pkg/sdkclient" sdkserverinfo "github.com/numaproj/numaflow/pkg/sdkclient/serverinfo" sinkclient "github.com/numaproj/numaflow/pkg/sdkclient/sinker" + "github.com/numaproj/numaflow/pkg/shared/callback" jsclient "github.com/numaproj/numaflow/pkg/shared/clients/nats" redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -66,6 +68,8 @@ func (u *SinkProcessor) Start(ctx context.Context) error { sinkHandler *udsink.UDSgRPCBasedUDSink fbSinkHandler *udsink.UDSgRPCBasedUDSink healthCheckers = make([]metrics.HealthChecker, 0) + vertexName = u.VertexInstance.Vertex.Spec.Name + pipelineName = u.VertexInstance.Vertex.Spec.PipelineName ) log := logging.FromContext(ctx) ctx, cancel := context.WithCancel(ctx) @@ -74,7 +78,7 @@ func (u *SinkProcessor) Start(ctx context.Context) error { // watermark variables no-op initialization // publishWatermark is a map representing a progressor per edge, we are initializing them to a no-op progressor // For sinks, the buffer name is the vertex name - fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{u.VertexInstance.Vertex.Spec.Name}) + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{vertexName}) idleManager = wmb.NewNoOpIdleManager() switch u.ISBSvcType { @@ -157,9 +161,6 @@ func (u *SinkProcessor) Start(ctx context.Context) error { } sinkHandler = udsink.NewUDSgRPCBasedUDSink(sdkClient) - if err != nil { - return fmt.Errorf("failed to create gRPC client, %w", err) - } // Readiness check if err := sinkHandler.WaitUntilReady(ctx); err != nil { return fmt.Errorf("failed on UDSink readiness check, %w", err) @@ -189,9 +190,6 @@ func (u *SinkProcessor) Start(ctx context.Context) error { } fbSinkHandler = udsink.NewUDSgRPCBasedUDSink(sdkClient) - if err != nil { - return fmt.Errorf("failed to create gRPC client, %w", err) - } // Readiness check if err = fbSinkHandler.WaitUntilReady(ctx); err != nil { return fmt.Errorf("failed on UDSink readiness check, %w", err) @@ -236,7 +234,19 @@ func (u *SinkProcessor) Start(ctx context.Context) error { forwardOpts = append(forwardOpts, sinkforward.WithFbSinkWriter(fbSinkWriter)) } - df, err := sinkforward.NewDataForward(u.VertexInstance, readers[index], sinkWriter, fetchWatermark, publishWatermark[u.VertexInstance.Vertex.Spec.Name], idleManager, forwardOpts...) + // if the callback is enabled, create a callback publisher + cbEnabled := sharedutil.LookupEnvBoolOr(dfv1.EnvCallbackEnabled, false) + if cbEnabled { + cbOpts := make([]callback.OptionFunc, 0) + cbUrl := os.Getenv(dfv1.EnvCallbackURL) + if cbUrl != "" { + cbOpts = append(cbOpts, callback.WithCallbackURL(cbUrl)) + } + cbPublisher := callback.NewUploader(ctx, vertexName, pipelineName, cbOpts...) + forwardOpts = append(forwardOpts, sinkforward.WithCallbackUploader(cbPublisher)) + } + + df, err := sinkforward.NewDataForward(u.VertexInstance, readers[index], sinkWriter, fetchWatermark, publishWatermark[vertexName], idleManager, forwardOpts...) if err != nil { return fmt.Errorf("failed to create data forward, error: %w", err) } diff --git a/pkg/sinks/udsink/sink.go b/pkg/sinks/udsink/sink.go index 7e67a831c4..36a7dc1219 100644 --- a/pkg/sinks/udsink/sink.go +++ b/pkg/sinks/udsink/sink.go @@ -65,7 +65,7 @@ func (s *UserDefinedSink) Write(ctx context.Context, messages []isb.Message) ([] msgs := make([]*sinkpb.SinkRequest, len(messages)) for i, m := range messages { msgs[i] = &sinkpb.SinkRequest{ - Id: m.ID, + Id: m.ID.String(), Value: m.Payload, Keys: m.Keys, EventTime: timestamppb.New(m.EventTime), diff --git a/pkg/sources/forward/data_forward.go b/pkg/sources/forward/data_forward.go index 328941db4b..cce95fb726 100644 --- a/pkg/sources/forward/data_forward.go +++ b/pkg/sources/forward/data_forward.go @@ -184,13 +184,6 @@ func (df *DataForward) Start() <-chan struct{} { return stopped } -// readWriteMessagePair represents a read message and its processed (via transformer) write messages. -type readWriteMessagePair struct { - readMessage *isb.ReadMessage - writeMessages []*isb.WriteMessage - transformerError error -} - // forwardAChunk forwards a chunk of message from the reader to the toBuffers. It does the Read -> Process -> Forward -> Ack chain // for a chunk of messages returned by the first Read call. It will return only if only we are successfully able to ack // the message after forwarding, barring any platform errors. The platform errors include buffer-full, @@ -288,12 +281,12 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { messageToStep[toVertex] = make([][]isb.Message, len(df.toBuffers[toVertex])) } - readWriteMessagePairs := make([]readWriteMessagePair, len(readMessages)) + readWriteMessagePairs := make([]isb.ReadWriteMessagePair, len(readMessages)) // If a user-defined transformer exists, apply it if df.opts.transformer != nil { // user-defined transformer concurrent processing request channel - transformerCh := make(chan *readWriteMessagePair) + transformerCh := make(chan *isb.ReadWriteMessagePair) // create a pool of Transformer Processors var wg sync.WaitGroup @@ -316,7 +309,7 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // assign watermark to the message m.Watermark = time.Time(processorWM) - readWriteMessagePairs[idx].readMessage = m + 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] @@ -349,10 +342,10 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // assign watermark to the message m.Watermark = time.Time(processorWM) - readWriteMessagePairs[idx].readMessage = m + readWriteMessagePairs[idx].ReadMessage = m // if no user-defined transformer exists, then the messages to write will be identical to the message read from source // thus, the unmodified read message will be stored as the corresponding writeMessage in readWriteMessagePairs - readWriteMessagePairs[idx].writeMessages = []*isb.WriteMessage{{Message: m.Message}} + readWriteMessagePairs[idx].WriteMessages = []*isb.WriteMessage{{Message: m.Message}} } } @@ -362,17 +355,17 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { latestEtMap := make(map[int32]int64) for _, m := range readWriteMessagePairs { - writeMessages = append(writeMessages, m.writeMessages...) - for _, message := range m.writeMessages { - message.Headers = m.readMessage.Headers + writeMessages = append(writeMessages, m.WriteMessages...) + for _, message := range m.WriteMessages { + message.Headers = m.ReadMessage.Headers // we convert each writeMessage to isb.ReadMessage by providing its parent ReadMessage's ReadOffset. // since we use message event time instead of the watermark to determine and publish source watermarks, // time.UnixMilli(-1) is assigned to the message watermark. transformedReadMessages are immediately // used below for publishing source watermarks. - if latestEt, ok := latestEtMap[m.readMessage.ReadOffset.PartitionIdx()]; !ok || message.EventTime.UnixNano() < latestEt { - latestEtMap[m.readMessage.ReadOffset.PartitionIdx()] = message.EventTime.UnixNano() + if latestEt, ok := latestEtMap[m.ReadMessage.ReadOffset.PartitionIdx()]; !ok || message.EventTime.UnixNano() < latestEt { + latestEtMap[m.ReadMessage.ReadOffset.PartitionIdx()] = message.EventTime.UnixNano() } - transformedReadMessages = append(transformedReadMessages, message.ToReadMessage(m.readMessage.ReadOffset, time.UnixMilli(-1))) + transformedReadMessages = append(transformedReadMessages, message.ToReadMessage(m.ReadMessage.ReadOffset, time.UnixMilli(-1))) } } @@ -395,7 +388,7 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { for _, m := range readWriteMessagePairs { // Look for errors in transformer processing if we see even 1 error we return. // Handling partial retrying is not worth ATM. - if m.transformerError != nil { + if m.Err != nil { metrics.SourceTransformerError.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, @@ -403,18 +396,18 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { metrics.LabelPartitionName: df.reader.GetName(), }).Inc() - df.opts.logger.Errorw("failed to apply source transformer", zap.Error(m.transformerError)) + df.opts.logger.Errorw("failed to apply source transformer", zap.Error(m.Err)) return } - // update toBuffers - for _, message := range m.writeMessages { - if err := df.whereToStep(message, messageToStep, m.readMessage); err != nil { + // for each message, we will determine where to send the message. + for _, message := range m.WriteMessages { + if err = df.whereToStep(message, messageToStep); err != nil { df.opts.logger.Errorw("failed in whereToStep", zap.Error(err)) return } } // get the list of source partitions for which we have read messages, we will use this to publish watermarks to toVertices - sourcePartitionsIndices[m.readMessage.ReadOffset.PartitionIdx()] = true + sourcePartitionsIndices[m.ReadMessage.ReadOffset.PartitionIdx()] = true } // forward the messages to the edge buffer (could be multiple edges) @@ -503,6 +496,12 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { metrics.LabelPartitionName: df.reader.GetName(), }).Add(float64(len(readOffsets))) + if df.opts.cbPublisher != nil { + if err := df.opts.cbPublisher.NonSinkVertexCallback(ctx, readWriteMessagePairs); err != nil { + df.opts.logger.Errorw("failed to publish callback", zap.Error(err)) + } + } + // ProcessingTimes of the entire forwardAChunk metrics.ForwardAChunkProcessingTime.With(map[string]string{ metrics.LabelVertex: df.vertexName, @@ -656,7 +655,7 @@ func (df *DataForward) writeToBuffer(ctx context.Context, toBufferPartition isb. } // concurrentApplyTransformer applies the transformer based on the request from the channel -func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessagePair <-chan *readWriteMessagePair) { +func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessagePair <-chan *isb.ReadWriteMessagePair) { for message := range readMessagePair { start := time.Now() metrics.SourceTransformerReadMessagesCount.With(map[string]string{ @@ -666,7 +665,7 @@ func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessa metrics.LabelPartitionName: df.reader.GetName(), }).Inc() - writeMessages, err := df.applyTransformer(ctx, message.readMessage) + writeMessages, err := df.applyTransformer(ctx, message.ReadMessage) metrics.SourceTransformerWriteMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, @@ -674,8 +673,8 @@ func (df *DataForward) concurrentApplyTransformer(ctx context.Context, readMessa metrics.LabelPartitionName: df.reader.GetName(), }).Add(float64(len(writeMessages))) - message.writeMessages = append(message.writeMessages, writeMessages...) - message.transformerError = err + message.WriteMessages = append(message.WriteMessages, writeMessages...) + message.Err = err metrics.SourceTransformerProcessingTime.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, @@ -710,24 +709,20 @@ func (df *DataForward) applyTransformer(ctx context.Context, readMessage *isb.Re return nil, err } continue - } else { - for index, m := range writeMessages { - m.ID = fmt.Sprintf("%s-%s-%d", readMessage.ReadOffset.String(), df.vertexName, index) - } - return writeMessages, nil } + return writeMessages, nil } } // whereToStep executes the WhereTo interfaces and then updates the to step's writeToBuffers buffer. -func (df *DataForward) whereToStep(writeMessage *isb.WriteMessage, messageToStep map[string][][]isb.Message, readMessage *isb.ReadMessage) error { +func (df *DataForward) whereToStep(writeMessage *isb.WriteMessage, messageToStep map[string][][]isb.Message) error { // call WhereTo and drop it on errors - to, err := df.toWhichStepDecider.WhereTo(writeMessage.Keys, writeMessage.Tags, writeMessage.ID) + to, err := df.toWhichStepDecider.WhereTo(writeMessage.Keys, writeMessage.Tags, writeMessage.ID.String()) if err != nil { df.opts.logger.Errorw("failed in whereToStep", zap.Error(isb.MessageWriteErr{ Name: df.reader.GetName(), - Header: readMessage.Header, - Body: readMessage.Body, + Header: writeMessage.Header, + Body: writeMessage.Body, Message: fmt.Sprintf("WhereTo failed, %s", err), })) @@ -750,8 +745,8 @@ func (df *DataForward) whereToStep(writeMessage *isb.WriteMessage, messageToStep if _, ok := messageToStep[t.ToVertexName]; !ok { df.opts.logger.Errorw("failed in whereToStep", zap.Error(isb.MessageWriteErr{ Name: df.reader.GetName(), - Header: readMessage.Header, - Body: readMessage.Body, + Header: writeMessage.Header, + Body: writeMessage.Body, Message: fmt.Sprintf("no such destination (%s)", t.ToVertexName), })) } diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index ad4ad596bd..25e41a9fa6 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -122,7 +122,7 @@ func (f myForwardTest) WhereTo(_ []string, _ []string, s string) ([]forwarder.Ve } func (f myForwardTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "test-vertex", message) } func TestNewDataForward(t *testing.T) { @@ -161,7 +161,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") fetchWatermark, _ := generic.BuildNoOpSourceWatermarkProgressorsFromBufferMap(toSteps) noOpStores := buildNoOpToVertexStores(toSteps) idleManager, _ := wmb.NewIdleManager(1, len(toSteps)) @@ -249,7 +249,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -400,7 +400,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -565,7 +565,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -714,7 +714,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") fetchWatermark, _ := generic.BuildNoOpSourceWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) @@ -748,7 +748,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -792,7 +792,7 @@ func TestNewDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -874,8 +874,11 @@ func (f mySourceForwardTest) ApplyTransform(ctx context.Context, message *isb.Re writeMessage := isb.Message{ Header: isb.Header{ MessageInfo: parentPaneInfo, - ID: offset.String(), - Keys: key, + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: offset.String(), + }, + Keys: key, }, Body: isb.Body{ Payload: result, @@ -916,7 +919,7 @@ func TestDataForwardSinglePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "testVertex") fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) @@ -937,7 +940,7 @@ func TestDataForwardSinglePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, int(count)) assert.Equal(t, []interface{}{writeMessages[0].Header.Keys, writeMessages[1].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"0-0-receivingVertex-0", "1-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{isb.MessageID{VertexName: "test-vertex", Offset: "0-0", Index: 0}, isb.MessageID{VertexName: "test-vertex", Offset: "1-0", Index: 0}}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) for _, m := range readMessages { // verify new event time gets assigned to messages. assert.Equal(t, testSourceNewEventTime, m.EventTime) @@ -971,7 +974,7 @@ func TestDataForwardMultiplePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "testVertex") fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) @@ -996,7 +999,18 @@ func TestDataForwardMultiplePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, 2) assert.Equal(t, []interface{}{writeMessages[0].Header.Keys, writeMessages[2].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"0-0-receivingVertex-0", "2-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{ + isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + Index: 0, + }, + isb.MessageID{ + VertexName: "test-vertex", + Offset: "2-0", + Index: 0, + }, + }, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) for _, m := range readMessages { // verify new event time gets assigned to messages. assert.Equal(t, testSourceNewEventTime, m.EventTime) @@ -1010,7 +1024,18 @@ func TestDataForwardMultiplePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, 2) assert.Equal(t, []interface{}{writeMessages[1].Header.Keys, writeMessages[3].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"1-0-receivingVertex-0", "3-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{ + isb.MessageID{ + VertexName: "test-vertex", + Offset: "1-0", + Index: 0, + }, + isb.MessageID{ + VertexName: "test-vertex", + Offset: "3-0", + Index: 0, + }, + }, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) for _, m := range readMessages { // verify new event time gets assigned to messages. assert.Equal(t, testSourceNewEventTime, m.EventTime) @@ -1107,7 +1132,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep = make(map[string][][]isb.Message) messageToStep["to1"] = make([][]isb.Message, 1) - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil, "testVertex") messageToStep["to1"][0] = append(messageToStep["to1"][0], writeMessages[0:value.batchSize+1]...) _, err = f.writeToBuffers(ctx, messageToStep) @@ -1129,7 +1154,7 @@ func (f myForwardDropTest) WhereTo(_ []string, _ []string, s string) ([]forwarde } func (f myForwardDropTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "test-vertex", message) } type myForwardToAllTest struct { @@ -1150,7 +1175,7 @@ func (f *myForwardToAllTest) WhereTo(_ []string, _ []string, s string) ([]forwar } func (f *myForwardToAllTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "test-vertex", message) } type myForwardInternalErrTest struct { @@ -1185,7 +1210,7 @@ func (f myForwardApplyWhereToErrTest) WhereTo(_ []string, _ []string, s string) } func (f myForwardApplyWhereToErrTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "test-vertex", message) } type myForwardApplyTransformerErrTest struct { diff --git a/pkg/sources/forward/options.go b/pkg/sources/forward/options.go index 1195ff4c46..f993855dee 100644 --- a/pkg/sources/forward/options.go +++ b/pkg/sources/forward/options.go @@ -22,6 +22,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/shared/callback" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/sources/forward/applier" ) @@ -38,6 +39,8 @@ type options struct { retryInterval time.Duration // logger is used to pass the logger variable logger *zap.SugaredLogger + // cbPublisher is the callback publisher for the vertex. + cbPublisher *callback.Uploader } type Option func(*options) error @@ -91,3 +94,11 @@ func WithTransformer(f applier.SourceTransformApplier) Option { return nil } } + +// WithCallbackUploader sets the callback uploader for the vertex +func WithCallbackUploader(cp *callback.Uploader) Option { + return func(o *options) error { + o.cbPublisher = cp + return nil + } +} diff --git a/pkg/sources/forward/shutdown_test.go b/pkg/sources/forward/shutdown_test.go index 0f9bbbf37b..a4ffc5e2e2 100644 --- a/pkg/sources/forward/shutdown_test.go +++ b/pkg/sources/forward/shutdown_test.go @@ -44,11 +44,11 @@ func (s myShutdownTest) WhereTo([]string, []string, string) ([]forwarder.VertexB } func (s myShutdownTest) ApplyTransform(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (s myShutdownTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } func TestInterStepDataForward(t *testing.T) { @@ -73,7 +73,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -112,7 +112,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/sources/generator/tickgen.go b/pkg/sources/generator/tickgen.go index 70a91b2d1e..713bc61ea4 100644 --- a/pkg/sources/generator/tickgen.go +++ b/pkg/sources/generator/tickgen.go @@ -308,8 +308,12 @@ func (mg *memGen) newReadMessage(key string, payload []byte, offset int64, et in Header: isb.Header{ // TODO: insert the right time based on the generator MessageInfo: isb.MessageInfo{EventTime: timeFromNanos(et, mg.jitter)}, - ID: readOffset.String(), - Keys: []string{key}, + ID: isb.MessageID{ + VertexName: mg.vertexName, + Offset: readOffset.String(), + Index: mg.vertexInstance.Replica, + }, + Keys: []string{key}, }, Body: isb.Body{Payload: payload}, } diff --git a/pkg/sources/http/http.go b/pkg/sources/http/http.go index 166329b3b8..2637764c93 100644 --- a/pkg/sources/http/http.go +++ b/pkg/sources/http/http.go @@ -157,8 +157,12 @@ func NewHttpSource(ctx context.Context, vertexInstance *dfv1.VertexInstance, opt Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{EventTime: eventTime}, - ID: id, - Headers: headers, + ID: isb.MessageID{ + VertexName: h.vertexName, + Offset: id, + Index: h.vertexReplica, + }, + Headers: headers, }, Body: isb.Body{ Payload: msg, diff --git a/pkg/sources/jetstream/jetstream.go b/pkg/sources/jetstream/jetstream.go index 66b2b7bfdb..ae1185fbc6 100644 --- a/pkg/sources/jetstream/jetstream.go +++ b/pkg/sources/jetstream/jetstream.go @@ -174,16 +174,32 @@ func New(ctx context.Context, vertexInstance *dfv1.VertexInstance, opts ...Optio n.logger.Errorw("Getting metadata for the message", zap.Error(err)) return } + + // Headers are supposed to be map[string][]string. However isb.Message.Header.Headers is map[string]string. + // So we only use the last header value in the slice. + headers := make(map[string]string, len(msg.Headers())) + for header, value := range msg.Headers() { + headers[header] = value[len(value)-1] + } + + readOffset := newOffset(msg, metadata.Sequence.Stream, inProgressTickDuration, n.logger) + m := &isb.ReadMessage{ Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{EventTime: metadata.Timestamp}, + ID: isb.MessageID{ + VertexName: n.vertexName, + Offset: readOffset.String(), + Index: readOffset.PartitionIdx(), + }, + Headers: headers, }, Body: isb.Body{ Payload: msg.Data(), }, }, - ReadOffset: newOffset(msg, metadata.Sequence.Stream, vertexInstance.Replica, inProgressTickDuration, n.logger), + ReadOffset: readOffset, } n.messages <- m } @@ -291,10 +307,10 @@ func (ns *jsSource) Pending(ctx context.Context) (int64, error) { } func (ns *jsSource) Ack(_ context.Context, offsets []isb.Offset) []error { - var errs []error - for _, o := range offsets { + errs := make([]error, len(offsets)) + for i, o := range offsets { if err := o.AckIt(); err != nil { - errs = append(errs, err) + errs[i] = err } } return errs diff --git a/pkg/sources/jetstream/offset.go b/pkg/sources/jetstream/offset.go index ce2ceba1fd..01d9539cc4 100644 --- a/pkg/sources/jetstream/offset.go +++ b/pkg/sources/jetstream/offset.go @@ -32,19 +32,17 @@ import ( // The implementation is same as `pkg/isb/stores/jetstream/reader.go` except for the type of `msg` field. // Once the ISB implementation starts using the new Jetstream client APIs, we can merge both. type offset struct { - msg jetstreamlib.Msg - seq uint64 - partitionID int32 - cancelFunc context.CancelFunc + msg jetstreamlib.Msg + seq uint64 + cancelFunc context.CancelFunc } var _ isb.Offset = (*offset)(nil) -func newOffset(msg jetstreamlib.Msg, seqNum uint64, partitionID int32, tickDuration time.Duration, log *zap.SugaredLogger) isb.Offset { +func newOffset(msg jetstreamlib.Msg, seqNum uint64, tickDuration time.Duration, log *zap.SugaredLogger) isb.Offset { o := &offset{ - msg: msg, - seq: seqNum, - partitionID: partitionID, + msg: msg, + seq: seqNum, } // If tickDuration is 1s, which means ackWait is 1s or 2s, it does not make much sense to do it, instead, increasing ackWait is recommended. if tickDuration.Seconds() > 1 { @@ -74,7 +72,7 @@ func (o *offset) workInProgress(ctx context.Context, msg jetstreamlib.Msg, tickD } func (o *offset) String() string { - return fmt.Sprintf("%d-%d", o.seq, o.partitionID) + return fmt.Sprintf("%d", o.seq) } func (o *offset) Sequence() (int64, error) { @@ -99,5 +97,6 @@ func (o *offset) NoAck() error { } func (o *offset) PartitionIdx() int32 { - return o.partitionID + // we only read from one stream/partition for a given JS Reader. + return 0 } diff --git a/pkg/sources/kafka/handler_test.go b/pkg/sources/kafka/handler_test.go index d3b9c7cf18..b5d4892312 100644 --- a/pkg/sources/kafka/handler_test.go +++ b/pkg/sources/kafka/handler_test.go @@ -26,6 +26,7 @@ import ( "github.com/stretchr/testify/assert" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/shared/logging" ) @@ -33,7 +34,7 @@ func TestMessageHandling(t *testing.T) { ctx := context.Background() topic := "testtopic" partition := int32(1) - offset := int64(1) + offset := fmt.Sprintf("%s:%d:%d", topic, partition, 1) value := "testvalue" keys := []string{"testkey"} @@ -67,12 +68,16 @@ func TestMessageHandling(t *testing.T) { msg := &sarama.ConsumerMessage{ Topic: topic, Partition: partition, - Offset: offset, + Offset: 1, Key: []byte(keys[0]), Value: []byte(value), } - expectedoffset := fmt.Sprintf("%s:%v:%v", topic, partition, offset) + expectedID := isb.MessageID{ + VertexName: vi.Vertex.Name, + Offset: fmt.Sprintf("%v", offset), + Index: partition, + } // push one message ks.handler.messages <- msg @@ -83,8 +88,8 @@ func TestMessageHandling(t *testing.T) { assert.Equal(t, 1, len(readmsgs)) readmsg := readmsgs[0] - assert.Equal(t, expectedoffset, readmsg.ID) + assert.Equal(t, expectedID, readmsg.ID) assert.Equal(t, []byte(value), readmsg.Body.Payload) assert.Equal(t, keys, readmsg.Header.Keys) - assert.Equal(t, expectedoffset, readmsg.ReadOffset.String()) + assert.Equal(t, expectedID.Offset, readmsg.ReadOffset.String()) } diff --git a/pkg/sources/kafka/reader.go b/pkg/sources/kafka/reader.go index ae3f179598..c397f52e92 100644 --- a/pkg/sources/kafka/reader.go +++ b/pkg/sources/kafka/reader.go @@ -330,9 +330,13 @@ func (ks *kafkaSource) toReadMessage(m *sarama.ConsumerMessage) *isb.ReadMessage msg := isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{EventTime: m.Timestamp}, - ID: readOffset.String(), - Keys: []string{string(m.Key)}, - Headers: headers, + ID: isb.MessageID{ + VertexName: ks.vertexName, + Offset: readOffset.String(), + Index: readOffset.PartitionIdx(), + }, + Keys: []string{string(m.Key)}, + Headers: headers, }, Body: body, } diff --git a/pkg/sources/nats/nats.go b/pkg/sources/nats/nats.go index 73cbbd3af1..46348f3635 100644 --- a/pkg/sources/nats/nats.go +++ b/pkg/sources/nats/nats.go @@ -127,7 +127,11 @@ func New(ctx context.Context, vertexInstance *dfv1.VertexInstance, opts ...Optio Header: isb.Header{ // TODO: Be able to specify event time. MessageInfo: isb.MessageInfo{EventTime: time.Now()}, - ID: readOffset.String(), + ID: isb.MessageID{ + VertexName: n.vertexName, + Offset: readOffset.String(), + Index: readOffset.PartitionIdx(), + }, }, Body: isb.Body{ Payload: msg.Data, diff --git a/pkg/sources/source.go b/pkg/sources/source.go index b8ee87656c..7c126a1f3b 100644 --- a/pkg/sources/source.go +++ b/pkg/sources/source.go @@ -19,6 +19,7 @@ package sources import ( "context" "fmt" + "os" "sync" "time" @@ -35,6 +36,7 @@ import ( sdkserverinfo "github.com/numaproj/numaflow/pkg/sdkclient/serverinfo" sourceclient "github.com/numaproj/numaflow/pkg/sdkclient/source" "github.com/numaproj/numaflow/pkg/sdkclient/sourcetransformer" + "github.com/numaproj/numaflow/pkg/shared/callback" jsclient "github.com/numaproj/numaflow/pkg/shared/clients/nats" redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -74,7 +76,7 @@ func (sp *SourceProcessor) Start(ctx context.Context) error { healthCheckers []metrics.HealthChecker idleManager wmb.IdleManager pipelineName = sp.VertexInstance.Vertex.Spec.PipelineName - vertexName = sp.VertexInstance.Vertex.Name + vertexName = sp.VertexInstance.Vertex.Spec.Name ) ctx, cancel := context.WithCancel(ctx) @@ -242,7 +244,7 @@ func (sp *SourceProcessor) Start(ctx context.Context) error { return fmt.Errorf("failed to create transformer gRPC client, %w", err) } - srcTransformerGRPCClient = transformer.NewGRPCBasedTransformer(srcTransformerClient) + srcTransformerGRPCClient = transformer.NewGRPCBasedTransformer(vertexName, srcTransformerClient) // Close the connection when we are done defer func() { @@ -269,6 +271,18 @@ func (sp *SourceProcessor) Start(ctx context.Context) error { // create a source watermark publisher sourceWmPublisher := publish.NewSourcePublish(ctx, pipelineName, vertexName, sourcePublisherStores, publish.WithDelay(sp.VertexInstance.Vertex.Spec.Watermark.GetMaxDelay())) + // if the callback is enabled, create a callback publisher + cbEnabled := sharedutil.LookupEnvBoolOr(dfv1.EnvCallbackEnabled, false) + if cbEnabled { + cbOpts := make([]callback.OptionFunc, 0) + cbUrl := os.Getenv(dfv1.EnvCallbackURL) + if cbUrl != "" { + cbOpts = append(cbOpts, callback.WithCallbackURL(cbUrl)) + } + cbPublisher := callback.NewUploader(ctx, vertexName, pipelineName, cbOpts...) + forwardOpts = append(forwardOpts, sourceforward.WithCallbackUploader(cbPublisher)) + } + // create source data forwarder var sourceForwarder *sourceforward.DataForward if sp.VertexInstance.Vertex.HasUDTransformer() { diff --git a/pkg/sources/transformer/grpc_transformer.go b/pkg/sources/transformer/grpc_transformer.go index e02d974564..14b414a348 100644 --- a/pkg/sources/transformer/grpc_transformer.go +++ b/pkg/sources/transformer/grpc_transformer.go @@ -35,12 +35,16 @@ import ( // GRPCBasedTransformer applies user-defined transformer over gRPC (over Unix Domain Socket) client/server where server is the transformer. type GRPCBasedTransformer struct { - client sourcetransformer.Client + vertexName string + client sourcetransformer.Client } // NewGRPCBasedTransformer returns a new gRPCBasedTransformer object. -func NewGRPCBasedTransformer(client sourcetransformer.Client) *GRPCBasedTransformer { - return &GRPCBasedTransformer{client: client} +func NewGRPCBasedTransformer(vertexName string, client sourcetransformer.Client) *GRPCBasedTransformer { + return &GRPCBasedTransformer{ + vertexName: vertexName, + client: client, + } } // IsHealthy checks if the transformer container is healthy. @@ -154,8 +158,12 @@ func (u *GRPCBasedTransformer) ApplyTransform(ctx context.Context, readMessage * Message: isb.Message{ Header: isb.Header{ MessageInfo: parentMessageInfo, - ID: fmt.Sprintf("%s-%d", offset.String(), i), - Keys: keys, + ID: isb.MessageID{ + VertexName: u.vertexName, + Offset: offset.String(), + Index: int32(i), + }, + Keys: keys, }, Body: isb.Body{ Payload: result.Value, diff --git a/pkg/sources/transformer/grpc_transformer_test.go b/pkg/sources/transformer/grpc_transformer_test.go index c74a5a88ea..959a40bf51 100644 --- a/pkg/sources/transformer/grpc_transformer_test.go +++ b/pkg/sources/transformer/grpc_transformer_test.go @@ -41,7 +41,7 @@ import ( func NewMockGRPCBasedTransformer(mockClient *transformermock.MockSourceTransformClient) *GRPCBasedTransformer { c, _ := sourcetransformer.NewFromClient(mockClient) - return &GRPCBasedTransformer{c} + return &GRPCBasedTransformer{"test-vertex", c} } func TestGRPCBasedTransformer_WaitUntilReadyWithMockClient(t *testing.T) { @@ -119,7 +119,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169600, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_success_key"}, }, Body: isb.Body{ @@ -163,7 +166,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ @@ -217,7 +223,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ @@ -267,7 +276,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ @@ -324,7 +336,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169720, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_success_key"}, }, Body: isb.Body{ @@ -367,7 +382,10 @@ func TestGRPCBasedTransformer_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ diff --git a/pkg/sources/udsource/grpc_udsource.go b/pkg/sources/udsource/grpc_udsource.go index 701521142f..525326651f 100644 --- a/pkg/sources/udsource/grpc_udsource.go +++ b/pkg/sources/udsource/grpc_udsource.go @@ -148,7 +148,7 @@ func (u *GRPCBasedUDSource) ApplyReadFn(ctx context.Context, count int64, timeou Message: isb.Message{ Header: isb.Header{ MessageInfo: isb.MessageInfo{EventTime: r.GetEventTime().AsTime()}, - ID: constructMessageID(offset.String(), r.GetOffset().GetPartitionId()), + ID: constructMessageID(u.vertexName, offset.String(), r.GetOffset().GetPartitionId()), Keys: r.GetKeys(), Headers: r.GetHeaders(), }, @@ -194,7 +194,11 @@ func (u *GRPCBasedUDSource) ApplyPartitionFn(ctx context.Context) ([]int32, erro return resp.GetResult().GetPartitions(), nil } -func constructMessageID(offset string, partitionIdx int32) string { +func constructMessageID(vertexName string, offset string, partitionIdx int32) isb.MessageID { // For a user-defined source, the partition ID plus the offset should be able to uniquely identify a message - return fmt.Sprintf("%d-%s", partitionIdx, offset) + return isb.MessageID{ + VertexName: vertexName, + Offset: offset, + Index: partitionIdx, + } } diff --git a/pkg/udf/forward/forward.go b/pkg/udf/forward/forward.go index 89b91349e9..bfc80576b0 100644 --- a/pkg/udf/forward/forward.go +++ b/pkg/udf/forward/forward.go @@ -181,13 +181,6 @@ func (isdf *InterStepDataForward) Start() <-chan struct{} { return stopped } -// readWriteMessagePair represents a read message and its processed (via map UDF) write messages. -type readWriteMessagePair struct { - readMessage *isb.ReadMessage - writeMessages []*isb.WriteMessage - udfError error -} - // forwardAChunk forwards a chunk of message from the fromBufferPartition to the toBuffers. It does the Read -> Process -> Forward -> Ack chain // for a chunk of messages returned by the first Read call. It will return only if only we are successfully able to ack // the message after forwarding, barring any platform errors. The platform errors include buffer-full, @@ -247,6 +240,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { } metrics.ReadDataMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(dataMessages))) metrics.ReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readMessages))) + udfResults := make([]isb.ReadWriteMessagePair, len(dataMessages)) // fetch watermark if available // TODO: make it async (concurrent and wait later) @@ -265,10 +259,9 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { } // udf concurrent processing request channel - udfCh := make(chan *readWriteMessagePair) + udfCh := make(chan *isb.ReadWriteMessagePair) // udfResults stores the results after map UDF processing for all read messages. It indexes // a read message to the corresponding write message - udfResults := make([]readWriteMessagePair, len(dataMessages)) // applyUDF, if there is an Internal error it is a blocking call and will return only if shutdown has been initiated. // create a pool of map UDF Processors @@ -289,7 +282,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // assign watermark to the message m.Watermark = time.Time(processorWM) // send map UDF processing work to the channel - udfResults[idx].readMessage = m + udfResults[idx].ReadMessage = m udfCh <- &udfResults[idx] } // let the go routines know that there is no more work @@ -306,16 +299,16 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { for _, m := range udfResults { // look for errors in udf processing, if we see even 1 error NoAck all messages // then return. Handling partial retrying is not worth ATM. - if m.udfError != nil { + if m.Err != nil { metrics.UDFError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() - isdf.opts.logger.Errorw("failed to applyUDF", zap.Error(m.udfError)) + isdf.opts.logger.Errorw("failed to applyUDF", zap.Error(m.Err)) // As there's no partial failure, non-ack all the readOffsets isdf.fromBufferPartition.NoAck(ctx, readOffsets) return } // update toBuffers - for _, message := range m.writeMessages { - if err := isdf.whereToStep(message, messageToStep, m.readMessage); err != nil { + for _, message := range m.WriteMessages { + if err := isdf.whereToStep(message, messageToStep, m.ReadMessage); err != nil { isdf.opts.logger.Errorw("failed in whereToStep", zap.Error(err)) isdf.fromBufferPartition.NoAck(ctx, readOffsets) return @@ -396,6 +389,12 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { } metrics.AckMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + if isdf.opts.cbPublisher != nil { + // Publish the callback for the vertex + if err = isdf.opts.cbPublisher.NonSinkVertexCallback(ctx, udfResults); err != nil { + isdf.opts.logger.Errorw("Failed to publish callback", zap.Error(err)) + } + } // ProcessingTimes of the entire forwardAChunk metrics.ForwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } @@ -445,8 +444,6 @@ func (isdf *InterStepDataForward) streamMessage( msgIndex := 0 for writeMessage := range writeMessageCh { writeMessage.Headers = dataMessages[0].Headers - // add vertex name to the ID, since multiple vertices can publish to the same vertex and we need uniqueness across them - writeMessage.ID = fmt.Sprintf("%s-%s-%d", dataMessages[0].ReadOffset.String(), isdf.vertexName, msgIndex) msgIndex += 1 metrics.UDFWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(1)) @@ -650,18 +647,18 @@ func (isdf *InterStepDataForward) writeToBuffer(ctx context.Context, toBufferPar } // concurrentApplyUDF applies the map UDF based on the request from the channel -func (isdf *InterStepDataForward) concurrentApplyUDF(ctx context.Context, readMessagePair <-chan *readWriteMessagePair) { +func (isdf *InterStepDataForward) concurrentApplyUDF(ctx context.Context, readMessagePair <-chan *isb.ReadWriteMessagePair) { for message := range readMessagePair { start := time.Now() metrics.UDFReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() - writeMessages, err := isdf.applyUDF(ctx, message.readMessage) + writeMessages, err := isdf.applyUDF(ctx, message.ReadMessage) metrics.UDFWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(writeMessages))) // set the headers for the write messages for _, m := range writeMessages { - m.Headers = message.readMessage.Headers + m.Headers = message.ReadMessage.Headers } - message.writeMessages = append(message.writeMessages, writeMessages...) - message.udfError = err + message.WriteMessages = append(message.WriteMessages, writeMessages...) + message.Err = err metrics.UDFProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } } @@ -685,20 +682,15 @@ func (isdf *InterStepDataForward) applyUDF(ctx context.Context, readMessage *isb return nil, err } continue - } else { - for index, m := range writeMessages { - // add vertex name to the ID, since multiple vertices can publish to the same vertex and we need uniqueness across them - m.ID = fmt.Sprintf("%s-%s-%d", readMessage.ReadOffset.String(), isdf.vertexName, index) - } - return writeMessages, nil } + return writeMessages, nil } } // whereToStep executes the WhereTo interfaces and then updates the to step's writeToBuffers buffer. func (isdf *InterStepDataForward) whereToStep(writeMessage *isb.WriteMessage, messageToStep map[string][][]isb.Message, readMessage *isb.ReadMessage) error { // call WhereTo and drop it on errors - to, err := isdf.FSD.WhereTo(writeMessage.Keys, writeMessage.Tags, writeMessage.ID) + to, err := isdf.FSD.WhereTo(writeMessage.Keys, writeMessage.Tags, writeMessage.ID.String()) if err != nil { isdf.opts.logger.Errorw("failed in whereToStep", zap.Error(isb.MessageWriteErr{Name: isdf.fromBufferPartition.GetName(), Header: readMessage.Header, Body: readMessage.Body, Message: fmt.Sprintf("WhereTo failed, %s", err)})) // a shutdown can break the blocking loop caused due to InternalErr diff --git a/pkg/udf/forward/forward_test.go b/pkg/udf/forward/forward_test.go index c9b0c35d46..fd40a5d3b8 100644 --- a/pkg/udf/forward/forward_test.go +++ b/pkg/udf/forward/forward_test.go @@ -90,11 +90,11 @@ func (f myForwardTest) WhereTo(_ []string, _ []string, s string) ([]forwarder.Ve } func (f myForwardTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (f myForwardTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } func TestNewInterStepDataForward(t *testing.T) { @@ -139,7 +139,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -216,7 +216,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -382,7 +382,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -560,7 +560,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -722,7 +722,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) idleManager, _ := wmb.NewIdleManager(1, len(toSteps)) @@ -754,7 +754,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -796,7 +796,7 @@ func TestNewInterStepDataForward(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -930,7 +930,7 @@ func TestNewInterStepDataForwardIdleWatermark(t *testing.T) { } ctrlMessage := []isb.Message{{Header: isb.Header{Kind: isb.WMB}}} - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "testVertex") fetchWatermark := &testWMBFetcher{WMBTestSameHeadWMB: true} toVertexWmStores := buildWatermarkStores(toSteps) @@ -1102,7 +1102,7 @@ func TestNewInterStepDataForwardIdleWatermark_Reset(t *testing.T) { Replica: 0, } - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "testVertex") fetchWatermark := &testWMBFetcher{WMBTestSameHeadWMB: true} toVertexWmStores := buildWatermarkStores(toSteps) @@ -1280,8 +1280,11 @@ func (f mySourceForwardTest) ApplyMap(ctx context.Context, message *isb.ReadMess writeMessage := isb.Message{ Header: isb.Header{ MessageInfo: parentPaneInfo, - ID: offset.String(), - Keys: key, + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: offset.String(), + }, + Keys: key, }, Body: isb.Body{ Payload: result, @@ -1309,8 +1312,11 @@ func (f mySourceForwardTest) ApplyMapStream(ctx context.Context, message *isb.Re writeMessage := isb.Message{ Header: isb.Header{ MessageInfo: parentPaneInfo, - ID: offset.String(), - Keys: key, + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: offset.String(), + }, + Keys: key, }, Body: isb.Body{ Payload: result, @@ -1339,7 +1345,7 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", AbstractVertex: dfv1.AbstractVertex{ - Name: "receivingVertex", + Name: "test-vertex", }, }} @@ -1350,7 +1356,7 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "test-vertex") fetchWatermark := &testForwardFetcher{} _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -1372,7 +1378,18 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, int(count)) assert.Equal(t, []interface{}{writeMessages[0].Header.Keys, writeMessages[1].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"0-0-receivingVertex-0", "1-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{ + isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + Index: 0, + }, + isb.MessageID{ + VertexName: "test-vertex", + Offset: "1-0", + Index: 0, + }, + }, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) assert.Equal(t, []interface{}{writeMessages[0].Header.Headers, writeMessages[1].Header.Headers}, []interface{}{readMessages[0].Header.Headers, readMessages[1].Header.Headers}) f.Stop() @@ -1392,7 +1409,7 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", AbstractVertex: dfv1.AbstractVertex{ - Name: "receivingVertex", + Name: "test-vertex", }, }} @@ -1403,7 +1420,7 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() - writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime, nil, "testVertex") fetchWatermark := &testForwardFetcher{} _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) @@ -1430,7 +1447,18 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, 2) assert.Equal(t, []interface{}{writeMessages[0].Header.Keys, writeMessages[2].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"0-0-receivingVertex-0", "2-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{ + isb.MessageID{ + VertexName: "test-vertex", + Offset: "0-0", + Index: 0, + }, + isb.MessageID{ + VertexName: "test-vertex", + Offset: "2-0", + Index: 0, + }, + }, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) assert.Equal(t, []interface{}{writeMessages[0].Header.Headers, writeMessages[2].Header.Headers}, []interface{}{readMessages[0].Header.Headers, readMessages[1].Header.Headers}) time.Sleep(time.Second) @@ -1439,7 +1467,18 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { assert.NoError(t, err, "expected no error") assert.Len(t, readMessages, 2) assert.Equal(t, []interface{}{writeMessages[1].Header.Keys, writeMessages[3].Header.Keys}, []interface{}{readMessages[0].Header.Keys, readMessages[1].Header.Keys}) - assert.Equal(t, []interface{}{"1-0-receivingVertex-0", "3-0-receivingVertex-0"}, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) + assert.Equal(t, []interface{}{ + isb.MessageID{ + VertexName: "test-vertex", + Offset: "1-0", + Index: 0, + }, + isb.MessageID{ + VertexName: "test-vertex", + Offset: "3-0", + Index: 0, + }, + }, []interface{}{readMessages[0].Header.ID, readMessages[1].Header.ID}) assert.Equal(t, []interface{}{writeMessages[1].Header.Headers, writeMessages[3].Header.Headers}, []interface{}{readMessages[0].Header.Headers, readMessages[1].Header.Headers}) f.Stop() @@ -1536,7 +1575,7 @@ func TestWriteToBuffer(t *testing.T) { // try to write to buffer after it is full. var messageToStep = make(map[string][][]isb.Message) messageToStep["to1"] = make([][]isb.Message, 1) - writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*value.batchSize, testStartTime, nil, "testVertex") messageToStep["to1"][0] = append(messageToStep["to1"][0], writeMessages[0:value.batchSize+1]...) _, err = f.writeToBuffers(ctx, messageToStep) @@ -1558,11 +1597,11 @@ func (f myForwardDropTest) WhereTo(_ []string, _ []string, s string) ([]forwarde } func (f myForwardDropTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (f myForwardDropTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } type myForwardToAllTest struct { @@ -1583,11 +1622,11 @@ func (f *myForwardToAllTest) WhereTo(_ []string, _ []string, s string) ([]forwar } func (f *myForwardToAllTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (f myForwardToAllTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } type myForwardInternalErrTest struct { @@ -1634,11 +1673,11 @@ func (f myForwardApplyWhereToErrTest) WhereTo(_ []string, _ []string, s string) } func (f myForwardApplyWhereToErrTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (f myForwardApplyWhereToErrTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } type myForwardApplyUDFErrTest struct { diff --git a/pkg/udf/forward/options.go b/pkg/udf/forward/options.go index 42d80848b9..35ac561868 100644 --- a/pkg/udf/forward/options.go +++ b/pkg/udf/forward/options.go @@ -22,6 +22,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/shared/callback" "github.com/numaproj/numaflow/pkg/shared/logging" ) @@ -37,6 +38,8 @@ type options struct { logger *zap.SugaredLogger // enableMapUdfStream indicates whether the message streaming is enabled or not for map UDF processing enableMapUdfStream bool + // cbPublisher is the callback publisher for the vertex. + cbPublisher *callback.Uploader } type Option func(*options) error @@ -54,7 +57,7 @@ func DefaultOptions() *options { // WithRetryInterval sets the retry interval func WithRetryInterval(f time.Duration) Option { return func(o *options) error { - o.retryInterval = time.Duration(f) + o.retryInterval = f return nil } } @@ -90,3 +93,11 @@ func WithUDFStreaming(f bool) Option { return nil } } + +// WithCallbackUploader sets the callback uploader for the vertex +func WithCallbackUploader(cp *callback.Uploader) Option { + return func(o *options) error { + o.cbPublisher = cp + return nil + } +} diff --git a/pkg/udf/forward/shutdown_test.go b/pkg/udf/forward/shutdown_test.go index 9ea2f923aa..5d0970194f 100644 --- a/pkg/udf/forward/shutdown_test.go +++ b/pkg/udf/forward/shutdown_test.go @@ -40,11 +40,11 @@ func (s myShutdownTest) WhereTo(_ []string, _ []string, _ string) ([]forwarder.V } func (s myShutdownTest) ApplyMap(ctx context.Context, message *isb.ReadMessage) ([]*isb.WriteMessage, error) { - return testutils.CopyUDFTestApply(ctx, message) + return testutils.CopyUDFTestApply(ctx, "", message) } func (s myShutdownTest) ApplyMapStream(ctx context.Context, message *isb.ReadMessage, writeMessageCh chan<- isb.WriteMessage) error { - return testutils.CopyUDFTestApplyStream(ctx, message, writeMessageCh) + return testutils.CopyUDFTestApplyStream(ctx, "", writeMessageCh, message) } func TestInterStepDataForward(t *testing.T) { @@ -76,7 +76,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", @@ -116,7 +116,7 @@ func TestInterStepDataForward(t *testing.T) { defer cancel() startTime := time.Unix(1636470000, 0) - writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil) + writeMessages := testutils.BuildTestWriteMessages(4*batchSize, startTime, nil, "testVertex") vertex := &dfv1.Vertex{Spec: dfv1.VertexSpec{ PipelineName: "testPipeline", diff --git a/pkg/udf/map_udf.go b/pkg/udf/map_udf.go index b7ea6bf64a..5c5d1f52ce 100644 --- a/pkg/udf/map_udf.go +++ b/pkg/udf/map_udf.go @@ -19,6 +19,7 @@ package udf import ( "context" "fmt" + "os" "sync" "go.uber.org/zap" @@ -31,6 +32,7 @@ import ( "github.com/numaproj/numaflow/pkg/sdkclient/mapper" "github.com/numaproj/numaflow/pkg/sdkclient/mapstreamer" sdkserverinfo "github.com/numaproj/numaflow/pkg/sdkclient/serverinfo" + "github.com/numaproj/numaflow/pkg/shared/callback" jsclient "github.com/numaproj/numaflow/pkg/shared/clients/nats" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" @@ -67,6 +69,8 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { mapHandler *rpc.GRPCBasedMap mapStreamHandler *rpc.GRPCBasedMapStream idleManager wmb.IdleManager + vertexName = u.VertexInstance.Vertex.Spec.Name + pipelineName = u.VertexInstance.Vertex.Spec.PipelineName ) // watermark variables @@ -125,12 +129,9 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { return fmt.Errorf("unrecognized isbsvc type %q", u.ISBSvcType) } - enableMapUdfStream, err := u.VertexInstance.Vertex.MapUdfStreamEnabled() - if err != nil { - return fmt.Errorf("failed to parse UDF map streaming metadata, %w", err) - } - maxMessageSize := sharedutil.LookupEnvIntOr(dfv1.EnvGRPCMaxMessageSize, sdkclient.DefaultGRPCMaxMessageSize) + + enableMapUdfStream := sharedutil.LookupEnvBoolOr(dfv1.EnvMapStreaming, false) if enableMapUdfStream { // Wait for server info to be ready serverInfo, err := sdkserverinfo.SDKServerInfo(sdkserverinfo.WithServerInfoFilePath(sdkclient.MapStreamServerInfoFile)) @@ -142,7 +143,7 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { if err != nil { return fmt.Errorf("failed to create map stream client, %w", err) } - mapStreamHandler = rpc.NewUDSgRPCBasedMapStream(mapStreamClient) + mapStreamHandler = rpc.NewUDSgRPCBasedMapStream(vertexName, mapStreamClient) // Readiness check if err := mapStreamHandler.WaitUntilReady(ctx); err != nil { @@ -166,7 +167,7 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { if err != nil { return fmt.Errorf("failed to create map client, %w", err) } - mapHandler = rpc.NewUDSgRPCBasedMap(mapClient) + mapHandler = rpc.NewUDSgRPCBasedMap(vertexName, mapClient) // Readiness check if err := mapHandler.WaitUntilReady(ctx); err != nil { @@ -236,6 +237,19 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { opts = append(opts, forward.WithUDFConcurrency(int(*x.ReadBatchSize))) } } + + // if the callback is enabled, create a callback publisher + cbEnabled := sharedutil.LookupEnvBoolOr(dfv1.EnvCallbackEnabled, false) + if cbEnabled { + cbOpts := make([]callback.OptionFunc, 0) + cbUrl := os.Getenv(dfv1.EnvCallbackURL) + if cbUrl != "" { + cbOpts = append(cbOpts, callback.WithCallbackURL(cbUrl)) + } + cbPublisher := callback.NewUploader(ctx, vertexName, pipelineName, cbOpts...) + opts = append(opts, forward.WithCallbackUploader(cbPublisher)) + } + // create a forwarder for each partition df, err := forward.NewInterStepDataForward(u.VertexInstance, readers[index], writers, conditionalForwarder, mapHandler, mapStreamHandler, fetchWatermark, publishWatermark, idleManager, opts...) if err != nil { diff --git a/pkg/udf/reduce_udf.go b/pkg/udf/reduce_udf.go index 0905ef171f..515d2207f8 100644 --- a/pkg/udf/reduce_udf.go +++ b/pkg/udf/reduce_udf.go @@ -141,7 +141,7 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { return fmt.Errorf("failed to create a new session reducer gRPC client: %w", err) } - reduceHandler := rpc.NewGRPCBasedUnalignedReduce(client) + reduceHandler := rpc.NewGRPCBasedUnalignedReduce(vertexName, client) // Readiness check if err := reduceHandler.WaitUntilReady(ctx); err != nil { return fmt.Errorf("failed on udf readiness check, %w", err) diff --git a/pkg/udf/rpc/grpc_aligned_reduce.go b/pkg/udf/rpc/grpc_aligned_reduce.go index 1a774306eb..a0952f8ecb 100644 --- a/pkg/udf/rpc/grpc_aligned_reduce.go +++ b/pkg/udf/rpc/grpc_aligned_reduce.go @@ -36,8 +36,6 @@ import ( "github.com/numaproj/numaflow/pkg/window" ) -// FIXME(session): rename file, type, NewXXX to Aligned - // GRPCBasedAlignedReduce is a reduce applier that uses gRPC client to invoke the aligned reduce UDF. It implements the applier.ReduceApplier interface. type GRPCBasedAlignedReduce struct { vertexName string @@ -112,7 +110,11 @@ func (u *GRPCBasedAlignedReduce) ApplyReduce(ctx context.Context, partitionID *p return } // create a unique message id for each response message which will be used for deduplication - msgId := fmt.Sprintf("%s-%d-%s-%d", u.vertexName, u.vertexReplica, partitionID.String(), index) + msgId := isb.MessageID{ + VertexName: u.vertexName, + Offset: fmt.Sprintf("%s-%d", partitionID.String(), u.vertexReplica), + Index: int32(index), + } index++ responseCh <- parseReduceResponse(result, msgId) case err := <-reduceErrCh: @@ -250,7 +252,7 @@ func convertToUdfError(err error) error { } // parseReduceResponse parse the SDK response to TimedWindowResponse -func parseReduceResponse(response *reducepb.ReduceResponse, msgId string) *window.TimedWindowResponse { +func parseReduceResponse(response *reducepb.ReduceResponse, msgId isb.MessageID) *window.TimedWindowResponse { taggedMessage := &isb.WriteMessage{ Message: isb.Message{ Header: isb.Header{ diff --git a/pkg/udf/rpc/grpc_map.go b/pkg/udf/rpc/grpc_map.go index 5b6251907d..2ace71e4d1 100644 --- a/pkg/udf/rpc/grpc_map.go +++ b/pkg/udf/rpc/grpc_map.go @@ -34,11 +34,15 @@ import ( // GRPCBasedMap is a map applier that uses gRPC client to invoke the map UDF. It implements the applier.MapApplier interface. type GRPCBasedMap struct { - client mapper.Client + vertexName string + client mapper.Client } -func NewUDSgRPCBasedMap(client mapper.Client) *GRPCBasedMap { - return &GRPCBasedMap{client: client} +func NewUDSgRPCBasedMap(vertexName string, client mapper.Client) *GRPCBasedMap { + return &GRPCBasedMap{ + vertexName: vertexName, + client: client, + } } // CloseConn closes the gRPC client connection. @@ -141,13 +145,18 @@ func (u *GRPCBasedMap) ApplyMap(ctx context.Context, readMessage *isb.ReadMessag } writeMessages := make([]*isb.WriteMessage, 0) - for _, result := range response.GetResults() { + for index, result := range response.GetResults() { keys := result.Keys taggedMessage := &isb.WriteMessage{ Message: isb.Message{ Header: isb.Header{ MessageInfo: parentMessageInfo, Keys: keys, + ID: isb.MessageID{ + VertexName: u.vertexName, + Offset: readMessage.ReadOffset.String(), + Index: int32(index), + }, }, Body: isb.Body{ Payload: result.Value, diff --git a/pkg/udf/rpc/grpc_map_test.go b/pkg/udf/rpc/grpc_map_test.go index ea8c4b36af..fc657a892b 100644 --- a/pkg/udf/rpc/grpc_map_test.go +++ b/pkg/udf/rpc/grpc_map_test.go @@ -59,7 +59,7 @@ func (r *rpcMsg) String() string { func NewMockUDSGRPCBasedMap(mockClient *mapmock.MockMapClient) *GRPCBasedMap { c, _ := mapper.NewFromClient(mockClient) - return &GRPCBasedMap{c} + return &GRPCBasedMap{"test-vertex", c} } func TestGRPCBasedMap_WaitUntilReadyWithMockClient(t *testing.T) { @@ -121,7 +121,10 @@ func TestGRPCBasedMap_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169600, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_success_key"}, }, Body: isb.Body{ @@ -173,7 +176,10 @@ func TestGRPCBasedMap_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ @@ -223,7 +229,10 @@ func TestGRPCBasedMap_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ @@ -280,7 +289,10 @@ func TestGRPCBasedMap_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169720, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_success_key"}, }, Body: isb.Body{ @@ -327,7 +339,10 @@ func TestGRPCBasedMap_BasicApplyWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ diff --git a/pkg/udf/rpc/grpc_mapstream.go b/pkg/udf/rpc/grpc_mapstream.go index 360422df17..bbbf7a4937 100644 --- a/pkg/udf/rpc/grpc_mapstream.go +++ b/pkg/udf/rpc/grpc_mapstream.go @@ -33,11 +33,15 @@ import ( // GRPCBasedMapStream is a map stream applier that uses gRPC client to invoke the map stream UDF. It implements the applier.MapStreamApplier interface. type GRPCBasedMapStream struct { - client mapstreamer.Client + vertexName string + client mapstreamer.Client } -func NewUDSgRPCBasedMapStream(client mapstreamer.Client) *GRPCBasedMapStream { - return &GRPCBasedMapStream{client: client} +func NewUDSgRPCBasedMapStream(vertexName string, client mapstreamer.Client) *GRPCBasedMapStream { + return &GRPCBasedMapStream{ + vertexName: vertexName, + client: client, + } } // CloseConn closes the gRPC client connection. @@ -111,8 +115,12 @@ func (u *GRPCBasedMapStream) ApplyMapStream(ctx context.Context, message *isb.Re Message: isb.Message{ Header: isb.Header{ MessageInfo: parentMessageInfo, - ID: fmt.Sprintf("%s-%d", offset.String(), i), - Keys: keys, + ID: isb.MessageID{ + VertexName: u.vertexName, + Offset: offset.String(), + Index: int32(i), + }, + Keys: keys, }, Body: isb.Body{ Payload: result.GetValue(), diff --git a/pkg/udf/rpc/grpc_mapstream_test.go b/pkg/udf/rpc/grpc_mapstream_test.go index 6329ec149f..148f9b5874 100644 --- a/pkg/udf/rpc/grpc_mapstream_test.go +++ b/pkg/udf/rpc/grpc_mapstream_test.go @@ -36,7 +36,7 @@ import ( func NewMockUDSGRPCBasedMapStream(mockClient *mapstreammock.MockMapStreamClient) *GRPCBasedMapStream { c, _ := mapstreamer.NewFromClient(mockClient) - return &GRPCBasedMapStream{c} + return &GRPCBasedMapStream{"test-vertex", c} } func TestGRPCBasedMapStream_WaitUntilReadyWithMockClient(t *testing.T) { @@ -104,7 +104,10 @@ func TestGRPCBasedUDF_BasicApplyStreamWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169600, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_success_key"}, }, Body: isb.Body{ @@ -167,7 +170,10 @@ func TestGRPCBasedUDF_BasicApplyStreamWithMockClient(t *testing.T) { MessageInfo: isb.MessageInfo{ EventTime: time.Unix(1661169660, 0), }, - ID: "test_id", + ID: isb.MessageID{ + VertexName: "test-vertex", + Offset: "test-offset", + }, Keys: []string{"test_error_key"}, }, Body: isb.Body{ diff --git a/pkg/udf/rpc/grpc_unaligned_reduce.go b/pkg/udf/rpc/grpc_unaligned_reduce.go index 604f7d4974..9660929010 100644 --- a/pkg/udf/rpc/grpc_unaligned_reduce.go +++ b/pkg/udf/rpc/grpc_unaligned_reduce.go @@ -18,6 +18,7 @@ package rpc import ( "context" + "errors" "fmt" "strconv" "strings" @@ -38,12 +39,14 @@ import ( // GRPCBasedUnalignedReduce is a reduce applier that uses gRPC client to invoke the session reduce UDF. It implements the applier.ReduceApplier interface. type GRPCBasedUnalignedReduce struct { + vertexName string client sessionreducer.Client resultsMap map[string]int } -func NewGRPCBasedUnalignedReduce(client sessionreducer.Client) *GRPCBasedUnalignedReduce { +func NewGRPCBasedUnalignedReduce(vertexName string, client sessionreducer.Client) *GRPCBasedUnalignedReduce { return &GRPCBasedUnalignedReduce{ + vertexName: vertexName, client: client, resultsMap: make(map[string]int), } @@ -110,7 +113,7 @@ func (u *GRPCBasedUnalignedReduce) ApplyReduce(ctx context.Context, partitionID case err := <-reduceErrCh: // ctx.Done() event will be handled by the AsyncReduceFn method // so we don't need a separate case for ctx.Done() here - if err == ctx.Err() { + if errors.Is(err, ctx.Err()) { errCh <- err return } @@ -254,9 +257,13 @@ func (u *GRPCBasedUnalignedReduce) parseSessionReduceResponse(response *sessionr } // updateMessageIDCount updates the message count in resultsMap and returns the updated message ID -func (u *GRPCBasedUnalignedReduce) updateAndGetMsgId(baseMsgId string) string { +func (u *GRPCBasedUnalignedReduce) updateAndGetMsgId(baseMsgId string) isb.MessageID { val := u.resultsMap[baseMsgId] val++ u.resultsMap[baseMsgId] = val - return fmt.Sprintf("%s:%d", baseMsgId, val) + return isb.MessageID{ + VertexName: u.vertexName, + Offset: baseMsgId, + Index: int32(val), + } } diff --git a/pkg/udf/rpc/grpc_unaligned_reduce_test.go b/pkg/udf/rpc/grpc_unaligned_reduce_test.go index a7d4d8eb58..1f3ea2a42b 100644 --- a/pkg/udf/rpc/grpc_unaligned_reduce_test.go +++ b/pkg/udf/rpc/grpc_unaligned_reduce_test.go @@ -39,7 +39,7 @@ import ( func NewMockUDSGRPCBasedSessionReduce(mockClient *sessionreducemock.MockSessionReduceClient) *GRPCBasedUnalignedReduce { c, _ := sessionreducer.NewFromClient(mockClient) rMap := make(map[string]int) - return &GRPCBasedUnalignedReduce{c, rMap} + return &GRPCBasedUnalignedReduce{"test-vertex", c, rMap} } func TestGRPCBasedSessionReduce_WaitUntilReady(t *testing.T) { diff --git a/test/api-e2e/api_test.go b/test/api-e2e/api_test.go index 76986cafb2..b99f60746a 100644 --- a/test/api-e2e/api_test.go +++ b/test/api-e2e/api_test.go @@ -3,6 +3,7 @@ package api_e2e import ( "context" "encoding/json" + "errors" "fmt" "strings" "testing" @@ -58,7 +59,7 @@ func (s *APISuite) TestISBSVC() { for !strings.Contains(getISBSVCBody, `"status":"healthy"`) { select { case <-ctx.Done(): - if ctx.Err() == context.DeadlineExceeded { + if errors.Is(ctx.Err(), context.DeadlineExceeded) { s.T().Fatalf("failed to get namespaces/isb-services: %v", ctx.Err()) } default: @@ -102,7 +103,7 @@ func (s *APISuite) TestISBSVCReplica1() { for !strings.Contains(getISBSVCBody, `"status":"healthy"`) { select { case <-ctx.Done(): - if ctx.Err() == context.DeadlineExceeded { + if errors.Is(ctx.Err(), context.DeadlineExceeded) { s.T().Fatalf("failed to get namespaces/isb-services: %v", ctx.Err()) } default: @@ -206,7 +207,7 @@ func (s *APISuite) TestPipeline1() { for strings.Contains(getPipelineISBsBody, "errMsg") { select { case <-ctx.Done(): - if ctx.Err() == context.DeadlineExceeded { + if errors.Is(ctx.Err(), context.DeadlineExceeded) { s.T().Fatalf("failed to get piplines/isbs: %v", ctx.Err()) } default: diff --git a/test/http-e2e/http_test.go b/test/http-e2e/http_test.go index 02ad31e5db..ce46e23aef 100644 --- a/test/http-e2e/http_test.go +++ b/test/http-e2e/http_test.go @@ -1,5 +1,3 @@ -//go:build test - /* Copyright 2022 The Numaproj Authors. @@ -22,8 +20,9 @@ import ( "fmt" "testing" - . "github.com/numaproj/numaflow/test/fixtures" "github.com/stretchr/testify/suite" + + . "github.com/numaproj/numaflow/test/fixtures" ) //go:generate kubectl apply -f testdata/http-auth-fake-secret.yaml -n numaflow-system