diff --git a/Makefile b/Makefile index aab918c933..373171265d 100644 --- a/Makefile +++ b/Makefile @@ -365,4 +365,4 @@ update-manifests-version: cat config/extensions/webhook/kustomization.yaml | sed 's/newTag: .*/newTag: $(VERSION)/' | sed 's@value: quay.io/numaproj/numaflow:.*@value: quay.io/numaproj/numaflow:$(VERSION)@' > /tmp/tmp_kustomization.yaml mv /tmp/tmp_kustomization.yaml config/extensions/webhook/kustomization.yaml cat Makefile | sed 's/^VERSION?=.*/VERSION?=$(VERSION)/' | sed 's/^BASE_VERSION:=.*/BASE_VERSION:=$(VERSION)/' > /tmp/ae_makefile - mv /tmp/ae_makefile Makefile + mv /tmp/ae_makefile Makefile \ No newline at end of file diff --git a/hack/generate-proto.sh b/hack/generate-proto.sh index ada70d6869..0e52c598b2 100755 --- a/hack/generate-proto.sh +++ b/hack/generate-proto.sh @@ -61,5 +61,5 @@ gen-protoc pkg/apis/proto/mvtxdaemon/mvtxdaemon.proto gen-protoc pkg/apis/proto/isb/message.proto -gen-protoc pkg/apis/proto/wmb/wmb.proto +gen-protoc pkg/apis/proto/watermark/watermark.proto diff --git a/pkg/apis/proto/watermark/watermark.pb.go b/pkg/apis/proto/watermark/watermark.pb.go new file mode 100644 index 0000000000..199e692e70 --- /dev/null +++ b/pkg/apis/proto/watermark/watermark.pb.go @@ -0,0 +1,265 @@ +// +//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. + +// Code generated by protoc-gen-go. DO NOT EDIT. +// versions: +// protoc-gen-go v1.34.2 +// protoc v5.27.2 +// source: pkg/apis/proto/watermark/watermark.proto + +package isb + +import ( + protoreflect "google.golang.org/protobuf/reflect/protoreflect" + protoimpl "google.golang.org/protobuf/runtime/protoimpl" + reflect "reflect" + sync "sync" +) + +const ( + // Verify that this generated code is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) + // Verify that runtime/protoimpl is sufficiently up-to-date. + _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) +) + +// WMB is used in the KV offset timeline bucket as the value for the given processor entity key. +type WMB struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Idle is set to true if the given processor entity hasn't published anything + // to the offset timeline bucket in a batch processing cycle. + // Idle is used to signal an idle watermark. + Idle bool `protobuf:"varint,1,opt,name=idle,proto3" json:"idle,omitempty"` + // Offset is the monotonically increasing index/offset of the buffer (buffer is the physical representation + // of the partition of the edge). + Offset int64 `protobuf:"varint,2,opt,name=offset,proto3" json:"offset,omitempty"` + // Watermark is tightly coupled with the offset and will be monotonically increasing for a given ProcessorEntity + // as the offset increases. + // When it is idling (Idle==true), for a given offset, the watermark can monotonically increase without offset + // increasing. + Watermark int64 `protobuf:"varint,3,opt,name=watermark,proto3" json:"watermark,omitempty"` + // Partition to identify the partition to which the watermark belongs. + Partition int32 `protobuf:"varint,4,opt,name=partition,proto3" json:"partition,omitempty"` +} + +func (x *WMB) Reset() { + *x = WMB{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_apis_proto_watermark_watermark_proto_msgTypes[0] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *WMB) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*WMB) ProtoMessage() {} + +func (x *WMB) ProtoReflect() protoreflect.Message { + mi := &file_pkg_apis_proto_watermark_watermark_proto_msgTypes[0] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use WMB.ProtoReflect.Descriptor instead. +func (*WMB) Descriptor() ([]byte, []int) { + return file_pkg_apis_proto_watermark_watermark_proto_rawDescGZIP(), []int{0} +} + +func (x *WMB) GetIdle() bool { + if x != nil { + return x.Idle + } + return false +} + +func (x *WMB) GetOffset() int64 { + if x != nil { + return x.Offset + } + return 0 +} + +func (x *WMB) GetWatermark() int64 { + if x != nil { + return x.Watermark + } + return 0 +} + +func (x *WMB) GetPartition() int32 { + if x != nil { + return x.Partition + } + return 0 +} + +// Heartbeat is used to track the active processors +type Heartbeat struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Heartbeat(current time in millis) published by the active processors. + Heartbeat int64 `protobuf:"varint,1,opt,name=heartbeat,proto3" json:"heartbeat,omitempty"` +} + +func (x *Heartbeat) Reset() { + *x = Heartbeat{} + if protoimpl.UnsafeEnabled { + mi := &file_pkg_apis_proto_watermark_watermark_proto_msgTypes[1] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *Heartbeat) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*Heartbeat) ProtoMessage() {} + +func (x *Heartbeat) ProtoReflect() protoreflect.Message { + mi := &file_pkg_apis_proto_watermark_watermark_proto_msgTypes[1] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use Heartbeat.ProtoReflect.Descriptor instead. +func (*Heartbeat) Descriptor() ([]byte, []int) { + return file_pkg_apis_proto_watermark_watermark_proto_rawDescGZIP(), []int{1} +} + +func (x *Heartbeat) GetHeartbeat() int64 { + if x != nil { + return x.Heartbeat + } + return 0 +} + +var File_pkg_apis_proto_watermark_watermark_proto protoreflect.FileDescriptor + +var file_pkg_apis_proto_watermark_watermark_proto_rawDesc = []byte{ + 0x0a, 0x28, 0x70, 0x6b, 0x67, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x2f, 0x77, 0x61, 0x74, 0x65, 0x72, + 0x6d, 0x61, 0x72, 0x6b, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x22, 0x6d, 0x0a, 0x03, 0x57, 0x4d, 0x42, 0x12, 0x12, 0x0a, 0x04, + 0x69, 0x64, 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x69, 0x64, 0x6c, 0x65, + 0x12, 0x16, 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, + 0x52, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x77, 0x61, 0x74, 0x65, + 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x77, 0x61, 0x74, + 0x65, 0x72, 0x6d, 0x61, 0x72, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, + 0x69, 0x6f, 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, + 0x74, 0x69, 0x6f, 0x6e, 0x22, 0x29, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, + 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x01, + 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x42, + 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x6e, 0x75, + 0x6d, 0x61, 0x70, 0x72, 0x6f, 0x6a, 0x2f, 0x6e, 0x75, 0x6d, 0x61, 0x66, 0x6c, 0x6f, 0x77, 0x2f, + 0x70, 0x6b, 0x67, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x69, + 0x73, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, +} + +var ( + file_pkg_apis_proto_watermark_watermark_proto_rawDescOnce sync.Once + file_pkg_apis_proto_watermark_watermark_proto_rawDescData = file_pkg_apis_proto_watermark_watermark_proto_rawDesc +) + +func file_pkg_apis_proto_watermark_watermark_proto_rawDescGZIP() []byte { + file_pkg_apis_proto_watermark_watermark_proto_rawDescOnce.Do(func() { + file_pkg_apis_proto_watermark_watermark_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_apis_proto_watermark_watermark_proto_rawDescData) + }) + return file_pkg_apis_proto_watermark_watermark_proto_rawDescData +} + +var file_pkg_apis_proto_watermark_watermark_proto_msgTypes = make([]protoimpl.MessageInfo, 2) +var file_pkg_apis_proto_watermark_watermark_proto_goTypes = []any{ + (*WMB)(nil), // 0: watermark.WMB + (*Heartbeat)(nil), // 1: watermark.Heartbeat +} +var file_pkg_apis_proto_watermark_watermark_proto_depIdxs = []int32{ + 0, // [0:0] is the sub-list for method output_type + 0, // [0:0] is the sub-list for method input_type + 0, // [0:0] is the sub-list for extension type_name + 0, // [0:0] is the sub-list for extension extendee + 0, // [0:0] is the sub-list for field type_name +} + +func init() { file_pkg_apis_proto_watermark_watermark_proto_init() } +func file_pkg_apis_proto_watermark_watermark_proto_init() { + if File_pkg_apis_proto_watermark_watermark_proto != nil { + return + } + if !protoimpl.UnsafeEnabled { + file_pkg_apis_proto_watermark_watermark_proto_msgTypes[0].Exporter = func(v any, i int) any { + switch v := v.(*WMB); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_pkg_apis_proto_watermark_watermark_proto_msgTypes[1].Exporter = func(v any, i int) any { + switch v := v.(*Heartbeat); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + } + type x struct{} + out := protoimpl.TypeBuilder{ + File: protoimpl.DescBuilder{ + GoPackagePath: reflect.TypeOf(x{}).PkgPath(), + RawDescriptor: file_pkg_apis_proto_watermark_watermark_proto_rawDesc, + NumEnums: 0, + NumMessages: 2, + NumExtensions: 0, + NumServices: 0, + }, + GoTypes: file_pkg_apis_proto_watermark_watermark_proto_goTypes, + DependencyIndexes: file_pkg_apis_proto_watermark_watermark_proto_depIdxs, + MessageInfos: file_pkg_apis_proto_watermark_watermark_proto_msgTypes, + }.Build() + File_pkg_apis_proto_watermark_watermark_proto = out.File + file_pkg_apis_proto_watermark_watermark_proto_rawDesc = nil + file_pkg_apis_proto_watermark_watermark_proto_goTypes = nil + file_pkg_apis_proto_watermark_watermark_proto_depIdxs = nil +} diff --git a/pkg/apis/proto/wmb/wmb.proto b/pkg/apis/proto/watermark/watermark.proto similarity index 88% rename from pkg/apis/proto/wmb/wmb.proto rename to pkg/apis/proto/watermark/watermark.proto index e77a4b7576..25e265766f 100644 --- a/pkg/apis/proto/wmb/wmb.proto +++ b/pkg/apis/proto/watermark/watermark.proto @@ -17,7 +17,7 @@ limitations under the License. syntax = "proto3"; option go_package = "github.com/numaproj/numaflow/pkg/apis/proto/isb"; -package wmb; +package watermark; // WMB is used in the KV offset timeline bucket as the value for the given processor entity key. message WMB { @@ -38,4 +38,10 @@ message WMB { // Partition to identify the partition to which the watermark belongs. int32 partition = 4; +} + +// Heartbeat is used to track the active processors +message Heartbeat { + // Heartbeat(current time in millis) published by the active processors. + int64 heartbeat = 1; } \ No newline at end of file diff --git a/pkg/apis/proto/wmb/wmb.pb.go b/pkg/apis/proto/wmb/wmb.pb.go deleted file mode 100644 index a11dc1c385..0000000000 --- a/pkg/apis/proto/wmb/wmb.pb.go +++ /dev/null @@ -1,199 +0,0 @@ -// -//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. - -// Code generated by protoc-gen-go. DO NOT EDIT. -// versions: -// protoc-gen-go v1.34.2 -// protoc v5.27.2 -// source: pkg/apis/proto/wmb/wmb.proto - -package isb - -import ( - protoreflect "google.golang.org/protobuf/reflect/protoreflect" - protoimpl "google.golang.org/protobuf/runtime/protoimpl" - reflect "reflect" - sync "sync" -) - -const ( - // Verify that this generated code is sufficiently up-to-date. - _ = protoimpl.EnforceVersion(20 - protoimpl.MinVersion) - // Verify that runtime/protoimpl is sufficiently up-to-date. - _ = protoimpl.EnforceVersion(protoimpl.MaxVersion - 20) -) - -// WMB is used in the KV offset timeline bucket as the value for the given processor entity key. -type WMB struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // Idle is set to true if the given processor entity hasn't published anything - // to the offset timeline bucket in a batch processing cycle. - // Idle is used to signal an idle watermark. - Idle bool `protobuf:"varint,1,opt,name=idle,proto3" json:"idle,omitempty"` - // Offset is the monotonically increasing index/offset of the buffer (buffer is the physical representation - // of the partition of the edge). - Offset int64 `protobuf:"varint,2,opt,name=offset,proto3" json:"offset,omitempty"` - // Watermark is tightly coupled with the offset and will be monotonically increasing for a given ProcessorEntity - // as the offset increases. - // When it is idling (Idle==true), for a given offset, the watermark can monotonically increase without offset - // increasing. - Watermark int64 `protobuf:"varint,3,opt,name=watermark,proto3" json:"watermark,omitempty"` - // Partition to identify the partition to which the watermark belongs. - Partition int32 `protobuf:"varint,4,opt,name=partition,proto3" json:"partition,omitempty"` -} - -func (x *WMB) Reset() { - *x = WMB{} - if protoimpl.UnsafeEnabled { - mi := &file_pkg_apis_proto_wmb_wmb_proto_msgTypes[0] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *WMB) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*WMB) ProtoMessage() {} - -func (x *WMB) ProtoReflect() protoreflect.Message { - mi := &file_pkg_apis_proto_wmb_wmb_proto_msgTypes[0] - if protoimpl.UnsafeEnabled && x != nil { - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - if ms.LoadMessageInfo() == nil { - ms.StoreMessageInfo(mi) - } - return ms - } - return mi.MessageOf(x) -} - -// Deprecated: Use WMB.ProtoReflect.Descriptor instead. -func (*WMB) Descriptor() ([]byte, []int) { - return file_pkg_apis_proto_wmb_wmb_proto_rawDescGZIP(), []int{0} -} - -func (x *WMB) GetIdle() bool { - if x != nil { - return x.Idle - } - return false -} - -func (x *WMB) GetOffset() int64 { - if x != nil { - return x.Offset - } - return 0 -} - -func (x *WMB) GetWatermark() int64 { - if x != nil { - return x.Watermark - } - return 0 -} - -func (x *WMB) GetPartition() int32 { - if x != nil { - return x.Partition - } - return 0 -} - -var File_pkg_apis_proto_wmb_wmb_proto protoreflect.FileDescriptor - -var file_pkg_apis_proto_wmb_wmb_proto_rawDesc = []byte{ - 0x0a, 0x1c, 0x70, 0x6b, 0x67, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x2f, 0x77, 0x6d, 0x62, 0x2f, 0x77, 0x6d, 0x62, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x03, - 0x77, 0x6d, 0x62, 0x22, 0x6d, 0x0a, 0x03, 0x57, 0x4d, 0x42, 0x12, 0x12, 0x0a, 0x04, 0x69, 0x64, - 0x6c, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x08, 0x52, 0x04, 0x69, 0x64, 0x6c, 0x65, 0x12, 0x16, - 0x0a, 0x06, 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, - 0x6f, 0x66, 0x66, 0x73, 0x65, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x77, 0x61, 0x74, 0x65, 0x72, 0x6d, - 0x61, 0x72, 0x6b, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x77, 0x61, 0x74, 0x65, 0x72, - 0x6d, 0x61, 0x72, 0x6b, 0x12, 0x1c, 0x0a, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, 0x6f, - 0x6e, 0x18, 0x04, 0x20, 0x01, 0x28, 0x05, 0x52, 0x09, 0x70, 0x61, 0x72, 0x74, 0x69, 0x74, 0x69, - 0x6f, 0x6e, 0x42, 0x31, 0x5a, 0x2f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x6e, 0x75, 0x6d, 0x61, 0x70, 0x72, 0x6f, 0x6a, 0x2f, 0x6e, 0x75, 0x6d, 0x61, 0x66, 0x6c, - 0x6f, 0x77, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x61, 0x70, 0x69, 0x73, 0x2f, 0x70, 0x72, 0x6f, 0x74, - 0x6f, 0x2f, 0x69, 0x73, 0x62, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, -} - -var ( - file_pkg_apis_proto_wmb_wmb_proto_rawDescOnce sync.Once - file_pkg_apis_proto_wmb_wmb_proto_rawDescData = file_pkg_apis_proto_wmb_wmb_proto_rawDesc -) - -func file_pkg_apis_proto_wmb_wmb_proto_rawDescGZIP() []byte { - file_pkg_apis_proto_wmb_wmb_proto_rawDescOnce.Do(func() { - file_pkg_apis_proto_wmb_wmb_proto_rawDescData = protoimpl.X.CompressGZIP(file_pkg_apis_proto_wmb_wmb_proto_rawDescData) - }) - return file_pkg_apis_proto_wmb_wmb_proto_rawDescData -} - -var file_pkg_apis_proto_wmb_wmb_proto_msgTypes = make([]protoimpl.MessageInfo, 1) -var file_pkg_apis_proto_wmb_wmb_proto_goTypes = []any{ - (*WMB)(nil), // 0: wmb.WMB -} -var file_pkg_apis_proto_wmb_wmb_proto_depIdxs = []int32{ - 0, // [0:0] is the sub-list for method output_type - 0, // [0:0] is the sub-list for method input_type - 0, // [0:0] is the sub-list for extension type_name - 0, // [0:0] is the sub-list for extension extendee - 0, // [0:0] is the sub-list for field type_name -} - -func init() { file_pkg_apis_proto_wmb_wmb_proto_init() } -func file_pkg_apis_proto_wmb_wmb_proto_init() { - if File_pkg_apis_proto_wmb_wmb_proto != nil { - return - } - if !protoimpl.UnsafeEnabled { - file_pkg_apis_proto_wmb_wmb_proto_msgTypes[0].Exporter = func(v any, i int) any { - switch v := v.(*WMB); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - } - type x struct{} - out := protoimpl.TypeBuilder{ - File: protoimpl.DescBuilder{ - GoPackagePath: reflect.TypeOf(x{}).PkgPath(), - RawDescriptor: file_pkg_apis_proto_wmb_wmb_proto_rawDesc, - NumEnums: 0, - NumMessages: 1, - NumExtensions: 0, - NumServices: 0, - }, - GoTypes: file_pkg_apis_proto_wmb_wmb_proto_goTypes, - DependencyIndexes: file_pkg_apis_proto_wmb_wmb_proto_depIdxs, - MessageInfos: file_pkg_apis_proto_wmb_wmb_proto_msgTypes, - }.Build() - File_pkg_apis_proto_wmb_wmb_proto = out.File - file_pkg_apis_proto_wmb_wmb_proto_rawDesc = nil - file_pkg_apis_proto_wmb_wmb_proto_goTypes = nil - file_pkg_apis_proto_wmb_wmb_proto_depIdxs = nil -} diff --git a/pkg/watermark/fetch/edge_fetcher_test.go b/pkg/watermark/fetch/edge_fetcher_test.go index 78f39b9ed9..91c0912382 100644 --- a/pkg/watermark/fetch/edge_fetcher_test.go +++ b/pkg/watermark/fetch/edge_fetcher_test.go @@ -26,7 +26,9 @@ import ( "github.com/nats-io/nats.go" "github.com/stretchr/testify/assert" "go.uber.org/zap/zaptest" + "google.golang.org/protobuf/proto" + wmbpb "github.com/numaproj/numaflow/pkg/apis/proto/watermark" "github.com/numaproj/numaflow/pkg/isb" natsclient "github.com/numaproj/numaflow/pkg/shared/clients/nats" natstest "github.com/numaproj/numaflow/pkg/shared/clients/nats/test" @@ -1693,7 +1695,12 @@ func manageHeartbeat(ctx context.Context, entityName string, hbStore kvs.KVStore start = !start default: if start { - _ = hbStore.PutKV(ctx, entityName, []byte(fmt.Sprintf("%d", time.Now().Unix()))) + hb := wmbpb.Heartbeat{Heartbeat: time.Now().Unix()} + marshal, err := proto.Marshal(&hb) + if err != nil { + return + } + _ = hbStore.PutKV(ctx, entityName, marshal) } time.Sleep(100 * time.Millisecond) } diff --git a/pkg/watermark/fetch/processor_manager.go b/pkg/watermark/fetch/processor_manager.go index 1dbeb99572..ba3ec2b82e 100644 --- a/pkg/watermark/fetch/processor_manager.go +++ b/pkg/watermark/fetch/processor_manager.go @@ -25,13 +25,14 @@ package fetch import ( "context" "fmt" - "strconv" "strings" "sync" "time" "go.uber.org/zap" + "google.golang.org/protobuf/proto" + wmbpb "github.com/numaproj/numaflow/pkg/apis/proto/watermark" "github.com/numaproj/numaflow/pkg/shared/kvs" "github.com/numaproj/numaflow/pkg/shared/logging" entity2 "github.com/numaproj/numaflow/pkg/watermark/entity" @@ -214,12 +215,13 @@ func (v *processorManager) startHeartBeatWatcher() { p.setStatus(_active) } // value is epoch - intValue, convErr := strconv.Atoi(string(value.Value())) + var hb wmbpb.Heartbeat + convErr := proto.Unmarshal(value.Value(), &hb) if convErr != nil { v.log.Errorw("Unable to convert intValue.WMB() to int64", zap.Error(convErr)) } else { // insert the last seen timestamp. we use this to figure whether this processor entity is inactive. - v.heartbeat.put(value.Key(), int64(intValue)) + v.heartbeat.put(value.Key(), hb.Heartbeat) } case kvs.KVDelete: p := v.getProcessor(value.Key()) diff --git a/pkg/watermark/publish/options.go b/pkg/watermark/publish/options.go index 4c3f69b683..3a27dd4211 100644 --- a/pkg/watermark/publish/options.go +++ b/pkg/watermark/publish/options.go @@ -34,9 +34,6 @@ type publishOptions struct { // Whether it is sink publisher or not // isSource and isSink should not be both true isSink bool - // partitionIdx is the default partition index - // It will be only used by source publisher - defaultPartitionIdx int32 } type PublishOption func(*publishOptions) @@ -73,10 +70,3 @@ func IsSink() PublishOption { opts.isSink = true } } - -// WithDefaultPartitionIdx sets the default partition index -func WithDefaultPartitionIdx(partitionIdx int32) PublishOption { - return func(opts *publishOptions) { - opts.defaultPartitionIdx = partitionIdx - } -} diff --git a/pkg/watermark/publish/publisher.go b/pkg/watermark/publish/publisher.go index cc558449ad..92525f6d5f 100644 --- a/pkg/watermark/publish/publisher.go +++ b/pkg/watermark/publish/publisher.go @@ -23,13 +23,14 @@ package publish import ( "context" - "fmt" "io" "sync" "time" "go.uber.org/zap" + "google.golang.org/protobuf/proto" + wmbpb "github.com/numaproj/numaflow/pkg/apis/proto/watermark" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/shared/kvs" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -273,7 +274,14 @@ func (p *publish) publishHeartbeat() { case <-p.ctx.Done(): return case <-ticker.C: - err := p.heartbeatStore.PutKV(p.ctx, p.entity.GetName(), []byte(fmt.Sprintf("%d", time.Now().Unix()))) + hbValue, err := proto.Marshal(&wmbpb.Heartbeat{ + Heartbeat: time.Now().Unix(), + }) + if err != nil { + p.log.Errorw("Unable to marshal heartbeat", zap.Error(err)) + continue + } + err = p.heartbeatStore.PutKV(p.ctx, p.entity.GetName(), hbValue) if err != nil { p.log.Errorw("put to bucket failed", zap.String("bucket", p.heartbeatStore.GetStoreName()), zap.Error(err)) } diff --git a/pkg/watermark/publish/source_publisher.go b/pkg/watermark/publish/source_publisher.go index 62f3e5950e..fdce4ce6db 100644 --- a/pkg/watermark/publish/source_publisher.go +++ b/pkg/watermark/publish/source_publisher.go @@ -51,8 +51,7 @@ type sourcePublish struct { // NewSourcePublish returns a new source publisher. func NewSourcePublish(ctx context.Context, pipelineName, vertexName string, srcPublishWMStores store.WatermarkStore, opts ...PublishOption) SourcePublisher { options := &publishOptions{ - defaultPartitionIdx: -1, - delay: 0, + delay: 0, } for _, opt := range opts { opt(options) @@ -66,18 +65,6 @@ func NewSourcePublish(ctx context.Context, pipelineName, vertexName string, srcP sourcePublishWMs: make(map[int32]Publisher), opts: options, } - - // if defaultPartitionIdx is set, create a publisher for it - // will be used by http, nats and tickgen source whose partitionIdx is same - // as the vertex replica index - if sp.opts.defaultPartitionIdx != -1 { - entityName := fmt.Sprintf("%s-%s-%d", sp.pipelineName, sp.vertexName, sp.opts.defaultPartitionIdx) - processorEntity := entity.NewProcessorEntity(entityName) - // toVertexPartitionCount is 1 because we publish watermarks within the source itself. - sourcePublishWM := NewPublish(sp.ctx, processorEntity, sp.srcPublishWMStores, 1, IsSource(), WithDelay(sp.opts.delay)) - sp.sourcePublishWMs[sp.opts.defaultPartitionIdx] = sourcePublishWM - } - return sp } diff --git a/pkg/watermark/wmb/wmb.go b/pkg/watermark/wmb/wmb.go index 6fbd306944..6cac099874 100644 --- a/pkg/watermark/wmb/wmb.go +++ b/pkg/watermark/wmb/wmb.go @@ -20,7 +20,7 @@ package wmb import ( "google.golang.org/protobuf/proto" - wmbpb "github.com/numaproj/numaflow/pkg/apis/proto/wmb" + wmbpb "github.com/numaproj/numaflow/pkg/apis/proto/watermark" ) // WMB is used in the KV offset timeline bucket as the value for the given processor entity key. diff --git a/rust/backoff/src/retry.rs b/rust/backoff/src/retry.rs index a0cc4b8eb7..9fc7196707 100644 --- a/rust/backoff/src/retry.rs +++ b/rust/backoff/src/retry.rs @@ -17,11 +17,11 @@ use crate::{Condition, Operation}; /// / /// (op) (Ok) -> [Return(Ok)]* /// \ / -/// (Ready) --- (Non-retriable) -> [Return(Err)]* +/// (Ready) --- (Non-retryable) -> [Return(Err)]* /// \ / /// (Err) (None) -> [Return(Err)]* /// \ / -/// (Retriable) ---> (Backoff) (Pending) +/// (Retryable) ---> (Backoff) (Pending) /// \ / /// (Sleep) /// \ diff --git a/rust/numaflow-core/src/config/monovertex.rs b/rust/numaflow-core/src/config/monovertex.rs index 4bd7ac5657..0c70b9f0e6 100644 --- a/rust/numaflow-core/src/config/monovertex.rs +++ b/rust/numaflow-core/src/config/monovertex.rs @@ -6,6 +6,8 @@ use base64::Engine; use numaflow_models::models::MonoVertex; use serde_json::from_slice; +use super::pipeline::ServingCallbackConfig; +use super::{DEFAULT_CALLBACK_CONCURRENCY, ENV_CALLBACK_CONCURRENCY, ENV_CALLBACK_ENABLED}; use crate::config::components::metrics::MetricsConfig; use crate::config::components::sink::SinkConfig; use crate::config::components::source::{GeneratorConfig, SourceConfig}; @@ -18,10 +20,6 @@ use crate::config::monovertex::sink::SinkType; use crate::error::Error; use crate::Result; -use super::pipeline::ServingCallbackConfig; - -use super::{DEFAULT_CALLBACK_CONCURRENCY, ENV_CALLBACK_CONCURRENCY, ENV_CALLBACK_ENABLED}; - const DEFAULT_BATCH_SIZE: u64 = 500; const DEFAULT_TIMEOUT_IN_MS: u32 = 1000; const DEFAULT_LOOKBACK_WINDOW_IN_SECS: u16 = 120; diff --git a/rust/numaflow-core/src/config/pipeline.rs b/rust/numaflow-core/src/config/pipeline.rs index aa3921b272..4ceb527642 100644 --- a/rust/numaflow-core/src/config/pipeline.rs +++ b/rust/numaflow-core/src/config/pipeline.rs @@ -6,21 +6,24 @@ use base64::prelude::BASE64_STANDARD; use base64::Engine; use numaflow_models::models::{ForwardConditions, Vertex}; use serde_json::from_slice; +use tracing::info; +use super::{DEFAULT_CALLBACK_CONCURRENCY, ENV_CALLBACK_CONCURRENCY, ENV_CALLBACK_ENABLED}; use crate::config::components::metrics::MetricsConfig; use crate::config::components::sink::SinkConfig; use crate::config::components::sink::SinkType; use crate::config::components::source::SourceConfig; use crate::config::components::transformer::{TransformerConfig, TransformerType}; use crate::config::get_vertex_replica; -use crate::config::pipeline::isb::{BufferReaderConfig, BufferWriterConfig}; +use crate::config::pipeline::isb::{BufferReaderConfig, BufferWriterConfig, Stream}; use crate::config::pipeline::map::MapMode; use crate::config::pipeline::map::MapVtxConfig; +use crate::config::pipeline::watermark::SourceWatermarkConfig; +use crate::config::pipeline::watermark::WatermarkConfig; +use crate::config::pipeline::watermark::{BucketConfig, EdgeWatermarkConfig}; use crate::error::Error; use crate::Result; -use super::{DEFAULT_CALLBACK_CONCURRENCY, ENV_CALLBACK_CONCURRENCY, ENV_CALLBACK_ENABLED}; - const DEFAULT_BATCH_SIZE: u64 = 500; const DEFAULT_TIMEOUT_IN_MS: u32 = 1000; const DEFAULT_LOOKBACK_WINDOW_IN_SECS: u16 = 120; @@ -35,11 +38,12 @@ pub(crate) const DEFAULT_STREAM_MAP_SOCKET: &str = "/var/run/numaflow/mapstream. const DEFAULT_MAP_SERVER_INFO_FILE: &str = "/var/run/numaflow/mapper-server-info"; pub(crate) mod isb; +pub(crate) mod watermark; #[derive(Debug, Clone, PartialEq)] pub(crate) struct PipelineConfig { - pub(crate) pipeline_name: String, - pub(crate) vertex_name: String, + pub(crate) pipeline_name: &'static str, + pub(crate) vertex_name: &'static str, pub(crate) replica: u16, pub(crate) batch_size: usize, // FIXME(cr): we cannot leak this as a paf, we need to use a different terminology. @@ -50,6 +54,7 @@ pub(crate) struct PipelineConfig { pub(crate) to_vertex_config: Vec, pub(crate) vertex_config: VertexType, pub(crate) metrics_config: MetricsConfig, + pub(crate) watermark_config: Option, pub(crate) callback_config: Option, } @@ -61,8 +66,8 @@ pub(crate) struct ServingCallbackConfig { impl Default for PipelineConfig { fn default() -> Self { PipelineConfig { - pipeline_name: "default-pl".to_string(), - vertex_name: "default-vtx".to_string(), + pipeline_name: Default::default(), + vertex_name: Default::default(), replica: 0, batch_size: DEFAULT_BATCH_SIZE as usize, paf_concurrency: (DEFAULT_BATCH_SIZE * 2) as usize, @@ -75,6 +80,7 @@ impl Default for PipelineConfig { transformer_config: None, }), metrics_config: Default::default(), + watermark_config: None, callback_config: None, } } @@ -130,7 +136,7 @@ pub(crate) mod map { impl TryFrom> for MapType { type Error = Error; - fn try_from(udf: Box) -> std::result::Result { + fn try_from(udf: Box) -> Result { if let Some(builtin) = udf.builtin { Ok(MapType::Builtin(BuiltinConfig { name: builtin.name, @@ -197,6 +203,7 @@ pub(crate) struct FromVertexConfig { #[derive(Debug, Clone, PartialEq)] pub(crate) struct ToVertexConfig { pub(crate) name: String, + pub(crate) partitions: u16, pub(crate) writer_config: BufferWriterConfig, pub(crate) conditions: Option>, } @@ -214,6 +221,8 @@ impl PipelineConfig { let vertex_obj: Vertex = from_slice(&decoded_spec) .map_err(|e| Error::Config(format!("Failed to parse pipeline spec: {:?}", e)))?; + info!("Loaded pipeline spec: {:?}", vertex_obj); + let pipeline_name = vertex_obj.spec.pipeline_name; let vertex_name = vertex_obj.spec.name; let replica = get_vertex_replica(); @@ -307,6 +316,8 @@ impl PipelineConfig { }) .collect(); + info!("Env vars found - {:#?}", env_vars); + let get_var = |var: &str| -> Result { Ok(env_vars .get(var) @@ -323,42 +334,49 @@ impl PipelineConfig { let mut from_vertex_config = vec![]; for edge in from_edges { let partition_count = edge.to_vertex_partition_count.unwrap_or_default() as u16; - let buffer_name = format!("{}-{}-{}", namespace, pipeline_name, edge.to); - let streams: Vec<(&'static str, u16)> = (0..partition_count) + let streams: Vec = (0..partition_count) .map(|i| { - let stream: &'static str = - Box::leak(Box::new(format!("{}-{}", buffer_name, i))); - (stream, i) + let ns: &'static str = Box::leak(namespace.clone().into_boxed_str()); + let pl: &'static str = Box::leak(pipeline_name.clone().into_boxed_str()); + let to: &'static str = Box::leak(edge.to.clone().into_boxed_str()); + let name: &'static str = + Box::leak(format!("{}-{}-{}-{}", ns, pl, to, i).into_boxed_str()); + Stream::new(name, to, i) }) .collect(); from_vertex_config.push(FromVertexConfig { - name: edge.from, + name: edge.from.clone(), reader_config: BufferReaderConfig { - partitions: partition_count, streams, ..Default::default() }, - partitions: 0, + partitions: partition_count, }); } let mut to_vertex_config = vec![]; for edge in to_edges { let partition_count = edge.to_vertex_partition_count.unwrap_or_default() as u16; - let buffer_name = format!("{}-{}-{}", namespace, pipeline_name, edge.to); - let streams: Vec<(String, u16)> = (0..partition_count) - .map(|i| (format!("{}-{}", buffer_name, i), i)) + let streams: Vec = (0..partition_count) + .map(|i| { + let ns: &'static str = Box::leak(namespace.clone().into_boxed_str()); + let pl: &'static str = Box::leak(pipeline_name.clone().into_boxed_str()); + let to: &'static str = Box::leak(edge.to.clone().into_boxed_str()); + let name: &'static str = + Box::leak(format!("{}-{}-{}-{}", ns, pl, to, i).into_boxed_str()); + Stream::new(name, to, i) + }) .collect(); let default_writer_config = BufferWriterConfig::default(); to_vertex_config.push(ToVertexConfig { - name: edge.to, + name: edge.to.clone(), + partitions: partition_count, writer_config: BufferWriterConfig { streams, - partitions: partition_count, max_length: edge .to_vertex_limits .as_ref() @@ -381,6 +399,23 @@ impl PipelineConfig { }); } + let watermark_config = if vertex_obj + .spec + .watermark + .map_or(true, |w| w.disabled.unwrap_or(true)) + { + Self::create_watermark_config( + &namespace, + &pipeline_name, + &vertex_name, + &vertex, + &from_vertex_config, + &to_vertex_config, + ) + } else { + None + }; + let look_back_window = vertex_obj .spec .scale @@ -389,7 +424,10 @@ impl PipelineConfig { .unwrap_or(DEFAULT_LOOKBACK_WINDOW_IN_SECS); let mut callback_config = None; - if get_var(ENV_CALLBACK_ENABLED).is_ok() { + if get_var(ENV_CALLBACK_ENABLED) + .map(|v| v == "true") + .unwrap_or(false) + { let callback_concurrency: usize = get_var(ENV_CALLBACK_CONCURRENCY) .unwrap_or_else(|_| format!("{DEFAULT_CALLBACK_CONCURRENCY}")) .parse() @@ -410,17 +448,114 @@ impl PipelineConfig { .parse() .unwrap(), read_timeout: Duration::from_millis(timeout_in_ms as u64), - pipeline_name, - vertex_name, + pipeline_name: Box::leak(pipeline_name.into_boxed_str()), + vertex_name: Box::leak(vertex_name.into_boxed_str()), replica: *replica, js_client_config, from_vertex_config, to_vertex_config, vertex_config: vertex, metrics_config: MetricsConfig::with_lookback_window_in_secs(look_back_window), + watermark_config, callback_config, }) } + + fn create_watermark_config( + namespace: &str, + pipeline_name: &str, + vertex_name: &str, + vertex: &VertexType, + from_vertex_config: &[FromVertexConfig], + to_vertex_config: &[ToVertexConfig], + ) -> Option { + match vertex { + VertexType::Source(_) => Some(WatermarkConfig::Source(SourceWatermarkConfig { + source_bucket_config: BucketConfig { + vertex: Box::leak(vertex_name.to_string().into_boxed_str()), + partitions: 1, // source will have only one partition + ot_bucket: Box::leak( + format!("{}-{}-{}_SOURCE_OT", namespace, pipeline_name, vertex_name) + .into_boxed_str(), + ), + hb_bucket: Box::leak( + format!( + "{}-{}-{}_SOURCE_PROCESSORS", + namespace, pipeline_name, vertex_name + ) + .into_boxed_str(), + ), + }, + to_vertex_bucket_config: to_vertex_config + .iter() + .map(|to| BucketConfig { + vertex: Box::leak(to.name.clone().into_boxed_str()), + partitions: to.partitions, + ot_bucket: Box::leak( + format!( + "{}-{}-{}-{}_OT", + namespace, pipeline_name, vertex_name, &to.name + ) + .into_boxed_str(), + ), + hb_bucket: Box::leak( + format!( + "{}-{}-{}-{}_PROCESSORS", + namespace, pipeline_name, vertex_name, &to.name + ) + .into_boxed_str(), + ), + }) + .collect(), + })), + VertexType::Sink(_) | VertexType::Map(_) => { + Some(WatermarkConfig::Edge(EdgeWatermarkConfig { + from_vertex_config: from_vertex_config + .iter() + .map(|from| BucketConfig { + vertex: Box::leak(from.name.clone().into_boxed_str()), + partitions: from.partitions, + ot_bucket: Box::leak( + format!( + "{}-{}-{}-{}_OT", + namespace, pipeline_name, &from.name, vertex_name + ) + .into_boxed_str(), + ), + hb_bucket: Box::leak( + format!( + "{}-{}-{}-{}_PROCESSORS", + namespace, pipeline_name, &from.name, vertex_name + ) + .into_boxed_str(), + ), + }) + .collect(), + to_vertex_config: to_vertex_config + .iter() + .map(|to| BucketConfig { + vertex: Box::leak(to.name.clone().into_boxed_str()), + partitions: to.partitions, + ot_bucket: Box::leak( + format!( + "{}-{}-{}-{}_OT", + namespace, pipeline_name, vertex_name, &to.name + ) + .into_boxed_str(), + ), + hb_bucket: Box::leak( + format!( + "{}-{}-{}-{}_PROCESSORS", + namespace, pipeline_name, vertex_name, &to.name + ) + .into_boxed_str(), + ), + }) + .collect(), + })) + } + } + } } #[cfg(test)] @@ -436,8 +571,8 @@ mod tests { #[test] fn test_default_pipeline_config() { let expected = PipelineConfig { - pipeline_name: "default-pl".to_string(), - vertex_name: "default-vtx".to_string(), + pipeline_name: Default::default(), + vertex_name: Default::default(), replica: 0, batch_size: DEFAULT_BATCH_SIZE as usize, paf_concurrency: (DEFAULT_BATCH_SIZE * 2) as usize, @@ -450,6 +585,7 @@ mod tests { transformer_config: None, }), metrics_config: Default::default(), + watermark_config: None, callback_config: None, }; @@ -483,8 +619,8 @@ mod tests { let pipeline_config = PipelineConfig::load(pipeline_cfg_base64, env_vars).unwrap(); let expected = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "out".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "out", replica: 0, batch_size: 500, paf_concurrency: 1000, @@ -497,11 +633,10 @@ mod tests { from_vertex_config: vec![FromVertexConfig { name: "in".to_string(), reader_config: BufferReaderConfig { - partitions: 1, - streams: vec![("default-simple-pipeline-out-0", 0)], + streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], wip_ack_interval: Duration::from_secs(1), }, - partitions: 0, + partitions: 1, }], to_vertex_config: vec![], vertex_config: VertexType::Sink(SinkVtxConfig { @@ -517,6 +652,15 @@ mod tests { lag_refresh_interval_in_secs: 3, lookback_window_in_secs: 120, }, + watermark_config: Some(WatermarkConfig::Edge(EdgeWatermarkConfig { + from_vertex_config: vec![BucketConfig { + vertex: "in", + partitions: 1, + ot_bucket: "default-simple-pipeline-in-out_OT", + hb_bucket: "default-simple-pipeline-in-out_PROCESSORS", + }], + to_vertex_config: vec![], + })), ..Default::default() }; assert_eq!(pipeline_config, expected); @@ -531,8 +675,8 @@ mod tests { PipelineConfig::load(pipeline_cfg_base64.to_string(), env_vars).unwrap(); let expected = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "in".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "in", replica: 0, batch_size: 1000, paf_concurrency: 1000, @@ -545,9 +689,9 @@ mod tests { from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { name: "out".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![("default-simple-pipeline-out-0".to_string(), 0)], - partitions: 1, + streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], max_length: 150000, usage_limit: 0.85, ..Default::default() @@ -569,6 +713,21 @@ mod tests { }, transformer_config: None, }), + metrics_config: Default::default(), + watermark_config: Some(WatermarkConfig::Source(SourceWatermarkConfig { + source_bucket_config: BucketConfig { + vertex: "in", + partitions: 1, + ot_bucket: "default-simple-pipeline-in_SOURCE_OT", + hb_bucket: "default-simple-pipeline-in_SOURCE_PROCESSORS", + }, + to_vertex_bucket_config: vec![BucketConfig { + vertex: "out", + partitions: 1, + ot_bucket: "default-simple-pipeline-in-out_OT", + hb_bucket: "default-simple-pipeline-in-out_PROCESSORS", + }], + })), ..Default::default() }; @@ -584,8 +743,8 @@ mod tests { PipelineConfig::load(pipeline_cfg_base64.to_string(), env_vars).unwrap(); let expected = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "in".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "in", replica: 0, batch_size: 50, paf_concurrency: 1000, @@ -598,9 +757,9 @@ mod tests { from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { name: "out".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![("default-simple-pipeline-out-0".to_string(), 0)], - partitions: 1, + streams: vec![Stream::new("default-simple-pipeline-out-0", "out", 0)], max_length: 30000, usage_limit: 0.8, ..Default::default() @@ -621,6 +780,21 @@ mod tests { }, transformer_config: None, }), + metrics_config: Default::default(), + watermark_config: Some(WatermarkConfig::Source(SourceWatermarkConfig { + source_bucket_config: BucketConfig { + vertex: "in", + partitions: 1, + ot_bucket: "default-simple-pipeline-in_SOURCE_OT", + hb_bucket: "default-simple-pipeline-in_SOURCE_PROCESSORS", + }, + to_vertex_bucket_config: vec![BucketConfig { + vertex: "out", + partitions: 1, + ot_bucket: "default-simple-pipeline-in-out_OT", + hb_bucket: "default-simple-pipeline-in-out_PROCESSORS", + }], + })), ..Default::default() }; @@ -707,8 +881,8 @@ mod tests { PipelineConfig::load(pipeline_cfg_base64.to_string(), env_vars).unwrap(); let expected = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "map".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "map", replica: 0, batch_size: 500, paf_concurrency: 1000, @@ -721,11 +895,10 @@ mod tests { from_vertex_config: vec![FromVertexConfig { name: "in".to_string(), reader_config: BufferReaderConfig { - partitions: 1, - streams: vec![("default-simple-pipeline-map-0", 0)], + streams: vec![Stream::new("default-simple-pipeline-map-0", "map", 0)], wip_ack_interval: Duration::from_secs(1), }, - partitions: 0, + partitions: 1, }], to_vertex_config: vec![], vertex_config: VertexType::Map(MapVtxConfig { @@ -737,6 +910,16 @@ mod tests { }), map_mode: MapMode::Unary, }), + metrics_config: MetricsConfig::default(), + watermark_config: Some(WatermarkConfig::Edge(EdgeWatermarkConfig { + from_vertex_config: vec![BucketConfig { + vertex: "in", + partitions: 1, + ot_bucket: "default-simple-pipeline-in-map_OT", + hb_bucket: "default-simple-pipeline-in-map_PROCESSORS", + }], + to_vertex_config: vec![], + })), ..Default::default() }; diff --git a/rust/numaflow-core/src/config/pipeline/isb.rs b/rust/numaflow-core/src/config/pipeline/isb.rs index 50da134e87..f258a6d5b3 100644 --- a/rust/numaflow-core/src/config/pipeline/isb.rs +++ b/rust/numaflow-core/src/config/pipeline/isb.rs @@ -1,9 +1,9 @@ -/// Jetstream ISB related configurations. +/// JetStream ISB related configurations. use std::fmt; +use std::fmt::Display; use std::time::Duration; const DEFAULT_PARTITION_IDX: u16 = 0; -const DEFAULT_PARTITIONS: u16 = 1; const DEFAULT_MAX_LENGTH: usize = 30000; const DEFAULT_USAGE_LIMIT: f64 = 0.8; const DEFAULT_BUFFER_FULL_STRATEGY: BufferFullStrategy = BufferFullStrategy::RetryUntilSuccess; @@ -29,20 +29,42 @@ pub(crate) mod jetstream { } } +/// Stream is a one of the partition of the ISB between two vertices. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct Stream { + pub(crate) name: &'static str, + pub(crate) vertex: &'static str, + pub(crate) partition: u16, +} + +impl Stream { + pub(crate) fn new(name: &'static str, vertex: &'static str, partition: u16) -> Self { + Stream { + name, + vertex, + partition, + } + } +} + +impl Display for Stream { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!(f, "{}", self.name) + } +} + #[derive(Debug, Clone, PartialEq)] pub(crate) struct BufferWriterConfig { - pub streams: Vec<(String, u16)>, - pub partitions: u16, - pub max_length: usize, - pub usage_limit: f64, - pub buffer_full_strategy: BufferFullStrategy, + pub(crate) streams: Vec, + pub(crate) max_length: usize, + pub(crate) usage_limit: f64, + pub(crate) buffer_full_strategy: BufferFullStrategy, } impl Default for BufferWriterConfig { fn default() -> Self { BufferWriterConfig { - streams: vec![("default-0".to_string(), DEFAULT_PARTITION_IDX)], - partitions: DEFAULT_PARTITIONS, + streams: vec![], max_length: DEFAULT_MAX_LENGTH, usage_limit: DEFAULT_USAGE_LIMIT, buffer_full_strategy: DEFAULT_BUFFER_FULL_STRATEGY, @@ -80,16 +102,14 @@ impl fmt::Display for BufferFullStrategy { #[derive(Debug, Clone, PartialEq)] pub(crate) struct BufferReaderConfig { - pub(crate) partitions: u16, - pub(crate) streams: Vec<(&'static str, u16)>, + pub(crate) streams: Vec, pub(crate) wip_ack_interval: Duration, } impl Default for BufferReaderConfig { fn default() -> Self { BufferReaderConfig { - partitions: DEFAULT_PARTITIONS, - streams: vec![("default-0", DEFAULT_PARTITION_IDX)], + streams: vec![Stream::new("default-0", "default", DEFAULT_PARTITION_IDX)], wip_ack_interval: Duration::from_millis(DEFAULT_WIP_ACK_INTERVAL_MILLIS), } } @@ -118,8 +138,7 @@ mod tests { #[test] fn test_default_buffer_writer_config() { let expected = BufferWriterConfig { - streams: vec![("default-0".to_string(), DEFAULT_PARTITION_IDX)], - partitions: DEFAULT_PARTITIONS, + streams: vec![], max_length: DEFAULT_MAX_LENGTH, usage_limit: DEFAULT_USAGE_LIMIT, buffer_full_strategy: DEFAULT_BUFFER_FULL_STRATEGY, @@ -141,8 +160,7 @@ mod tests { #[test] fn test_default_buffer_reader_config() { let expected = BufferReaderConfig { - partitions: DEFAULT_PARTITIONS, - streams: vec![("default-0", DEFAULT_PARTITION_IDX)], + streams: vec![Stream::new("default-0", "default", DEFAULT_PARTITION_IDX)], wip_ack_interval: Duration::from_millis(DEFAULT_WIP_ACK_INTERVAL_MILLIS), }; let config = BufferReaderConfig::default(); diff --git a/rust/numaflow-core/src/config/pipeline/watermark.rs b/rust/numaflow-core/src/config/pipeline/watermark.rs new file mode 100644 index 0000000000..dbfb7cf109 --- /dev/null +++ b/rust/numaflow-core/src/config/pipeline/watermark.rs @@ -0,0 +1,33 @@ +/// Watermark config for different types of Vertex. +#[derive(Debug, Clone, PartialEq)] +pub(crate) enum WatermarkConfig { + Source(SourceWatermarkConfig), + Edge(EdgeWatermarkConfig), +} + +/// Source's Watermark configuration is different because it does publish/fetch/publish because +/// Watermark starts at Source. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct SourceWatermarkConfig { + pub(crate) source_bucket_config: BucketConfig, + pub(crate) to_vertex_bucket_config: Vec, +} + +/// Watermark movements are captured via a Key/Value bucket. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct BucketConfig { + pub(crate) vertex: &'static str, + pub(crate) partitions: u16, + /// Offset Timeline (OT) bucket. + pub(crate) ot_bucket: &'static str, + /// Heartbeat bucket for processor heartbeats. + pub(crate) hb_bucket: &'static str, +} + +/// Edge's Watermark is purely based on the previous vertex and the next vertex. It only has to +/// implement fetch/publish. +#[derive(Debug, Clone, PartialEq)] +pub(crate) struct EdgeWatermarkConfig { + pub(crate) from_vertex_config: Vec, + pub(crate) to_vertex_config: Vec, +} diff --git a/rust/numaflow-core/src/error.rs b/rust/numaflow-core/src/error.rs index 0e499d0689..fe6f7f1916 100644 --- a/rust/numaflow-core/src/error.rs +++ b/rust/numaflow-core/src/error.rs @@ -55,6 +55,9 @@ pub enum Error { #[error("Task Error - {0}")] Tracker(String), + + #[error("Watermark Error - {0}")] + Watermark(String), } impl From for Error { diff --git a/rust/numaflow-core/src/lib.rs b/rust/numaflow-core/src/lib.rs index c9665ca780..86f64d0a91 100644 --- a/rust/numaflow-core/src/lib.rs +++ b/rust/numaflow-core/src/lib.rs @@ -51,9 +51,17 @@ mod pipeline; /// Tracker to track the completeness of message processing. mod tracker; -/// Map is a feature that allows users to execute custom code to transform their data. +/// [Map] is a feature that allows users to execute custom code to transform their data. +/// +/// [Map]: https://numaflow.numaproj.io/user-guide/user-defined-functions/map/map/ mod mapper; +/// [Watermark] _is a monotonically increasing timestamp of the oldest work/event not yet completed_ +/// +/// +/// [Watermark]: https://numaflow.numaproj.io/core-concepts/watermarks/ +mod watermark; + pub async fn run() -> Result<()> { let cln_token = CancellationToken::new(); let shutdown_cln_token = cln_token.clone(); diff --git a/rust/numaflow-core/src/mapper/map.rs b/rust/numaflow-core/src/mapper/map.rs index 5969744374..d8682b0620 100644 --- a/rust/numaflow-core/src/mapper/map.rs +++ b/rust/numaflow-core/src/mapper/map.rs @@ -1,7 +1,6 @@ use std::sync::Arc; use std::time::Duration; -use bytes::Bytes; use numaflow_pb::clients::map::map_client::MapClient; use tokio::sync::{mpsc, oneshot, OwnedSemaphorePermit, Semaphore}; use tokio::task::JoinHandle; @@ -15,7 +14,7 @@ use crate::error::Error; use crate::mapper::map::user_defined::{ UserDefinedBatchMap, UserDefinedStreamMap, UserDefinedUnaryMap, }; -use crate::message::Message; +use crate::message::{Message, Offset}; use crate::tracker::TrackerHandle; pub(super) mod user_defined; @@ -328,7 +327,7 @@ impl MapHandle { tokio::spawn(async move { let _permit = permit; - let offset = read_msg.id.offset.clone(); + let offset = read_msg.offset.clone(); let (sender, receiver) = oneshot::channel(); let msg = UnaryActorMessage { message: read_msg.clone(), @@ -403,13 +402,13 @@ impl MapHandle { for receiver in receivers { match receiver.await { Ok(Ok(mapped_messages)) => { - let mut offset: Option = None; + let mut offset: Option = None; for message in mapped_messages.iter() { if offset.is_none() { - offset = Some(message.id.offset.clone()); + offset = Some(message.offset.clone()); } tracker_handle - .update(message.id.offset.clone(), message.tags.clone()) + .update(message.offset.clone(), message.tags.clone()) .await?; } if let Some(offset) = offset { @@ -471,10 +470,7 @@ impl MapHandle { match result { Ok(mapped_message) => { if let Err(e) = tracker_handle - .update( - mapped_message.id.offset.clone(), - mapped_message.tags.clone(), - ) + .update(mapped_message.offset.clone(), mapped_message.tags.clone()) .await { error_tx.send(e).await.expect("failed to send error"); @@ -495,7 +491,7 @@ impl MapHandle { } } - if let Err(e) = tracker_handle.update_eof(read_msg.id.offset).await { + if let Err(e) = tracker_handle.update_eof(read_msg.offset).await { error_tx.send(e).await.expect("failed to send error"); } }); @@ -550,7 +546,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( @@ -567,11 +563,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(crate::message::StringOffset::new( - "0".to_string(), - 0, - ))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -643,7 +637,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( MapMode::Unary, @@ -663,8 +657,9 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("value_{}", i).into(), - offset: Some(Offset::String(StringOffset::new(i.to_string(), 0))), + offset: Offset::String(StringOffset::new(i.to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: i.to_string().into(), @@ -734,7 +729,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( MapMode::Unary, @@ -753,8 +748,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -794,7 +790,7 @@ mod tests { impl batchmap::BatchMapper for SimpleBatchMap { async fn batchmap( &self, - mut input: tokio::sync::mpsc::Receiver, + mut input: mpsc::Receiver, ) -> Vec { let mut responses: Vec = Vec::new(); while let Some(datum) = input.recv().await { @@ -812,7 +808,7 @@ mod tests { #[tokio::test] async fn batch_mapper_operations() -> Result<()> { - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let (shutdown_tx, shutdown_rx) = oneshot::channel(); let tmp_dir = TempDir::new().unwrap(); let sock_file = tmp_dir.path().join("batch_map.sock"); let server_info_file = tmp_dir.path().join("batch_map-server-info"); @@ -830,7 +826,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( @@ -848,8 +844,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -862,8 +859,9 @@ mod tests { keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), - offset: Some(Offset::String(StringOffset::new("1".to_string(), 1))), + offset: Offset::String(StringOffset::new("1".to_string(), 1)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "1".to_string().into(), @@ -943,7 +941,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( MapMode::Batch, @@ -960,8 +958,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -974,8 +973,9 @@ mod tests { keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), - offset: Some(Offset::String(StringOffset::new("1".to_string(), 1))), + offset: Offset::String(StringOffset::new("1".to_string(), 1)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "1".to_string().into(), @@ -1055,7 +1055,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = MapClient::new(create_rpc_channel(sock_file).await?); let mapper = MapHandle::new( @@ -1072,8 +1072,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "test,map,stream".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -1154,7 +1155,7 @@ mod tests { tokio::time::sleep(Duration::from_millis(100)).await; let client = MapClient::new(create_rpc_channel(sock_file).await?); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let mapper = MapHandle::new( MapMode::Stream, 500, @@ -1169,8 +1170,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "panic".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), diff --git a/rust/numaflow-core/src/mapper/map/user_defined.rs b/rust/numaflow-core/src/mapper/map/user_defined.rs index c879843563..b8d71bd5e2 100644 --- a/rust/numaflow-core/src/mapper/map/user_defined.rs +++ b/rust/numaflow-core/src/mapper/map/user_defined.rs @@ -33,11 +33,11 @@ impl From for MapRequest { request: Some(map::map_request::Request { keys: message.keys.to_vec(), value: message.value.to_vec(), - event_time: prost_timestamp_from_utc(message.event_time), - watermark: None, + event_time: Some(prost_timestamp_from_utc(message.event_time)), + watermark: message.watermark.map(prost_timestamp_from_utc), headers: message.headers, }), - id: message.offset.unwrap().to_string(), + id: message.offset.to_string(), handshake: None, status: None, } @@ -114,9 +114,9 @@ impl UserDefinedUnaryMap { message: Message, respond_to: oneshot::Sender>>, ) { - let key = message.offset.clone().unwrap().to_string(); + let key = message.offset.clone().to_string(); let msg_info = ParentMessageInfo { - offset: message.offset.clone().expect("offset can never be none"), + offset: message.offset.clone(), event_time: message.event_time, headers: message.headers.clone(), }; @@ -212,9 +212,9 @@ impl UserDefinedBatchMap { respond_to: Vec>>>, ) { for (message, respond_to) in messages.into_iter().zip(respond_to) { - let key = message.offset.clone().unwrap().to_string(); + let key = message.offset.clone().to_string(); let msg_info = ParentMessageInfo { - offset: message.offset.clone().expect("offset can never be none"), + offset: message.offset.clone(), event_time: message.event_time, headers: message.headers.clone(), }; @@ -258,9 +258,10 @@ async fn process_response(sender_map: &ResponseSenderMap, resp: MapResponse) { keys: Arc::from(result.keys), tags: Some(Arc::from(result.tags)), value: result.value.into(), - offset: Some(msg_info.offset.clone()), + offset: msg_info.offset.clone(), event_time: msg_info.event_time, headers: msg_info.headers.clone(), + watermark: None, metadata: None, }; response_messages.push(message); @@ -385,9 +386,10 @@ impl UserDefinedStreamMap { keys: Arc::from(result.keys), tags: Some(Arc::from(result.tags)), value: result.value.into(), - offset: None, + offset: message_info.offset.clone(), event_time: message_info.event_time, headers: message_info.headers.clone(), + watermark: None, metadata: None, }; response_sender @@ -411,9 +413,9 @@ impl UserDefinedStreamMap { message: Message, respond_to: mpsc::Sender>, ) { - let key = message.offset.clone().unwrap().to_string(); + let key = message.offset.clone().to_string(); let msg_info = ParentMessageInfo { - offset: message.offset.clone().expect("offset can never be none"), + offset: message.offset.clone(), event_time: message.event_time, headers: message.headers.clone(), }; @@ -487,11 +489,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(crate::message::Offset::String(StringOffset::new( - "0".to_string(), - 0, - ))), + offset: crate::message::Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -578,11 +578,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(crate::message::Offset::String(StringOffset::new( - "0".to_string(), - 0, - ))), + offset: crate::message::Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -595,11 +593,9 @@ mod tests { keys: Arc::from(vec!["second".into()]), tags: None, value: "world".into(), - offset: Some(crate::message::Offset::String(StringOffset::new( - "1".to_string(), - 1, - ))), + offset: crate::message::Offset::String(StringOffset::new("1".to_string(), 1)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "1".to_string().into(), @@ -695,11 +691,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "test,map,stream".into(), - offset: Some(crate::message::Offset::String(StringOffset::new( - "0".to_string(), - 0, - ))), + offset: crate::message::Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), diff --git a/rust/numaflow-core/src/message.rs b/rust/numaflow-core/src/message.rs index 37ac9b2c57..8ebd90788f 100644 --- a/rust/numaflow-core/src/message.rs +++ b/rust/numaflow-core/src/message.rs @@ -9,7 +9,6 @@ use prost::Message as ProtoMessage; use serde::{Deserialize, Serialize}; use crate::shared::grpc::prost_timestamp_from_utc; -use crate::shared::grpc::utc_from_timestamp; use crate::Error; const DROP: &str = "U+005C__DROP__"; @@ -27,9 +26,11 @@ pub(crate) struct Message { /// offset of the message, it is optional because offset is only /// available when we read the message, and we don't persist the /// offset in the ISB. - pub(crate) offset: Option, + pub(crate) offset: Offset, /// event time of the message pub(crate) event_time: DateTime, + /// watermark of the message + pub(crate) watermark: Option>, /// id of the message pub(crate) id: MessageID, /// headers of the message @@ -38,6 +39,22 @@ pub(crate) struct Message { pub(crate) metadata: Option, } +impl Default for Message { + fn default() -> Self { + Self { + keys: Arc::new([]), + tags: None, + value: Bytes::new(), + offset: Default::default(), + event_time: Utc::now(), + watermark: None, + id: Default::default(), + headers: HashMap::new(), + metadata: None, + } + } +} + #[derive(Debug, Clone)] pub(crate) struct Metadata { /// name of the previous vertex. @@ -46,7 +63,7 @@ pub(crate) struct Metadata { } /// Offset of the message which will be used to acknowledge the message. -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +#[derive(Debug, Clone, Hash, Serialize, Deserialize, Eq, PartialEq)] pub(crate) enum Offset { Int(IntOffset), String(StringOffset), @@ -61,24 +78,30 @@ impl fmt::Display for Offset { } } +impl Default for Offset { + fn default() -> Self { + Offset::Int(Default::default()) + } +} + impl Message { // Check if the message should be dropped. pub(crate) fn dropped(&self) -> bool { self.tags .as_ref() - .map_or(false, |tags| tags.contains(&DROP.to_string())) + .is_some_and(|tags| tags.contains(&DROP.to_string())) } } /// IntOffset is integer based offset enum type. -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] -pub struct IntOffset { - pub(crate) offset: u64, +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash, Default)] +pub(crate) struct IntOffset { + pub(crate) offset: i64, pub(crate) partition_idx: u16, } impl IntOffset { - pub fn new(seq: u64, partition_idx: u16) -> Self { + pub(crate) fn new(seq: i64, partition_idx: u16) -> Self { Self { offset: seq, partition_idx, @@ -93,7 +116,7 @@ impl fmt::Display for IntOffset { } /// StringOffset is string based offset enum type. -#[derive(Debug, Clone, Serialize, Deserialize, PartialEq)] +#[derive(Debug, Clone, Serialize, Deserialize, PartialEq, Eq, Hash)] pub(crate) struct StringOffset { /// offset could be a complex base64 string. pub(crate) offset: Bytes, @@ -136,6 +159,16 @@ pub(crate) struct MessageID { pub(crate) index: i32, } +impl Default for MessageID { + fn default() -> Self { + Self { + vertex_name: Bytes::new(), + offset: Bytes::new(), + index: 0, + } + } +} + impl From for MessageID { fn from(id: numaflow_pb::objects::isb::MessageId) -> Self { Self { @@ -174,7 +207,7 @@ impl TryFrom for BytesMut { let proto_message = numaflow_pb::objects::isb::Message { header: Some(numaflow_pb::objects::isb::Header { message_info: Some(numaflow_pb::objects::isb::MessageInfo { - event_time: prost_timestamp_from_utc(message.event_time), + event_time: Some(prost_timestamp_from_utc(message.event_time)), is_late: false, // Set this according to your logic }), kind: numaflow_pb::objects::isb::MessageKind::Data as i32, @@ -195,37 +228,6 @@ impl TryFrom for BytesMut { } } -impl TryFrom for Message { - type Error = Error; - - fn try_from(bytes: Bytes) -> Result { - let proto_message = numaflow_pb::objects::isb::Message::decode(bytes) - .map_err(|e| Error::Proto(e.to_string()))?; - - let header = proto_message - .header - .ok_or(Error::Proto("Missing header".to_string()))?; - let body = proto_message - .body - .ok_or(Error::Proto("Missing body".to_string()))?; - let message_info = header - .message_info - .ok_or(Error::Proto("Missing message_info".to_string()))?; - let id = header.id.ok_or(Error::Proto("Missing id".to_string()))?; - - Ok(Message { - keys: Arc::from(header.keys.into_boxed_slice()), - tags: None, - value: body.payload.into(), - offset: None, - event_time: utc_from_timestamp(message_info.event_time), - id: id.into(), - headers: header.headers, - metadata: None, - }) - } -} - #[cfg(test)] mod tests { use std::collections::HashMap; @@ -263,11 +265,12 @@ mod tests { keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), - offset: Some(Offset::String(StringOffset { + offset: Offset::String(StringOffset { offset: "123".to_string().into(), partition_idx: 0, - })), + }), event_time: Utc.timestamp_opt(1627846261, 0).unwrap(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "123".to_string().into(), @@ -283,7 +286,7 @@ mod tests { let proto_message = ProtoMessage { header: Some(Header { message_info: Some(MessageInfo { - event_time: prost_timestamp_from_utc(message.event_time), + event_time: Some(prost_timestamp_from_utc(message.event_time)), is_late: false, }), kind: numaflow_pb::objects::isb::MessageKind::Data as i32, @@ -301,44 +304,6 @@ mod tests { assert_eq!(result.unwrap(), buf); } - #[test] - fn test_vec_u8_to_message() { - let proto_message = ProtoMessage { - header: Some(Header { - message_info: Some(MessageInfo { - event_time: prost_timestamp_from_utc(Utc.timestamp_opt(1627846261, 0).unwrap()), - is_late: false, - }), - kind: numaflow_pb::objects::isb::MessageKind::Data as i32, - id: Some(MessageId { - vertex_name: "vertex".to_string(), - offset: "123".to_string(), - index: 0, - }), - keys: vec!["key1".to_string()], - headers: HashMap::new(), - }), - body: Some(Body { - payload: vec![1, 2, 3], - }), - }; - - let mut buf = BytesMut::new(); - prost::Message::encode(&proto_message, &mut buf).unwrap(); - let buf = buf.freeze(); - - let result: Result = buf.try_into(); - assert!(result.is_ok()); - - let message = result.unwrap(); - assert_eq!(message.keys.to_vec(), vec!["key1".to_string()]); - assert_eq!(message.value, vec![1, 2, 3]); - assert_eq!( - message.event_time, - Utc.timestamp_opt(1627846261, 0).unwrap() - ); - } - #[test] fn test_message_id_from_proto() { let proto_id = MessageId { diff --git a/rust/numaflow-core/src/metrics.rs b/rust/numaflow-core/src/metrics.rs index e0eb0cc1e6..46bf45fd68 100644 --- a/rust/numaflow-core/src/metrics.rs +++ b/rust/numaflow-core/src/metrics.rs @@ -29,7 +29,7 @@ use tonic::Request; use tracing::{debug, error, info}; use crate::config::{get_pipeline_name, get_vertex_name, get_vertex_replica}; -use crate::pipeline::isb::jetstream::reader::JetstreamReader; +use crate::pipeline::isb::jetstream::reader::JetStreamReader; use crate::source::Source; use crate::Error; @@ -722,7 +722,7 @@ struct TimestampedPending { pub(crate) enum LagReader { Source(Source), #[allow(clippy::upper_case_acronyms)] - ISB(Vec), // multiple partitions + ISB(Vec), // multiple partitions } /// PendingReader is responsible for periodically checking the lag of the reader @@ -925,7 +925,7 @@ async fn fetch_source_pending(lag_reader: &Source) -> crate::error::Result Ok(response) } -async fn fetch_isb_pending(reader: &mut JetstreamReader) -> crate::error::Result { +async fn fetch_isb_pending(reader: &mut JetStreamReader) -> crate::error::Result { let response: i64 = reader.pending().await?.map_or(-1, |p| p as i64); // default to -1(unavailable) Ok(response) } diff --git a/rust/numaflow-core/src/monovertex.rs b/rust/numaflow-core/src/monovertex.rs index b829156a70..92ca072287 100644 --- a/rust/numaflow-core/src/monovertex.rs +++ b/rust/numaflow-core/src/monovertex.rs @@ -28,7 +28,8 @@ pub(crate) async fn start_forwarder( .callback_config .as_ref() .map(|cb_cfg| CallbackHandler::new(config.name.clone(), cb_cfg.callback_concurrency)); - let tracker_handle = TrackerHandle::new(callback_handler); + let tracker_handle = TrackerHandle::new(None, callback_handler); + let (transformer, transformer_grpc_client) = create_components::create_transformer( config.batch_size, config.transformer_config.clone(), @@ -43,6 +44,7 @@ pub(crate) async fn start_forwarder( &config.source_config, tracker_handle.clone(), transformer, + None, cln_token.clone(), ) .await?; diff --git a/rust/numaflow-core/src/monovertex/forwarder.rs b/rust/numaflow-core/src/monovertex/forwarder.rs index 9aeecf2f8e..44a8ae13ae 100644 --- a/rust/numaflow-core/src/monovertex/forwarder.rs +++ b/rust/numaflow-core/src/monovertex/forwarder.rs @@ -13,12 +13,16 @@ //! ``` //! //! Most of the data move forward except for the `ack` which can happen only after the that the tracker -//! has guaranteed that the processing complete. +//! has guaranteed that the processing complete. Ack is spawned during the reading. //! ```text //! (Read) +-------> (UDF) -------> (Write) + //! | | //! | | -//! +-------> {Ack} <----------------+ +//! +-------> {tracker} <------------ +//! | +//! | +//! v +//! {ack} //! //! {} -> Listens on a OneShot //! () -> Streaming Interface @@ -188,7 +192,7 @@ mod tests { #[tokio::test] async fn test_forwarder() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); // create the source which produces x number of messages let cln_token = CancellationToken::new(); @@ -245,13 +249,14 @@ mod tests { .await .map_err(|e| panic!("failed to create source reader: {:?}", e)) .unwrap(); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let source = Source::new( 5, SourceType::UserDefinedSource(src_read, src_ack, lag_reader), tracker_handle.clone(), true, Some(transformer), + None, ); let sink_writer = SinkWriterBuilder::new( @@ -318,7 +323,7 @@ mod tests { #[tokio::test] async fn test_flatmap_operation() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); // create the source which produces x number of messages let cln_token = CancellationToken::new(); @@ -381,6 +386,7 @@ mod tests { tracker_handle.clone(), true, Some(transformer), + None, ); let sink_writer = SinkWriterBuilder::new( diff --git a/rust/numaflow-core/src/pipeline.rs b/rust/numaflow-core/src/pipeline.rs index 652f1c8181..e787ae1df5 100644 --- a/rust/numaflow-core/src/pipeline.rs +++ b/rust/numaflow-core/src/pipeline.rs @@ -7,18 +7,24 @@ use serving::callback::CallbackHandler; use tokio_util::sync::CancellationToken; use tracing::info; +use crate::config::is_mono_vertex; +use crate::config::pipeline; +use crate::config::pipeline::isb::Stream; use crate::config::pipeline::map::MapVtxConfig; +use crate::config::pipeline::watermark::WatermarkConfig; use crate::config::pipeline::{PipelineConfig, SinkVtxConfig, SourceVtxConfig}; -use crate::config::{is_mono_vertex, pipeline}; use crate::metrics::{LagReader, PipelineContainerState, UserDefinedContainerState}; use crate::pipeline::forwarder::source_forwarder; -use crate::pipeline::isb::jetstream::reader::JetstreamReader; +use crate::pipeline::isb::jetstream::reader::JetStreamReader; use crate::pipeline::isb::jetstream::writer::JetstreamWriter; use crate::pipeline::pipeline::isb::BufferReaderConfig; use crate::shared::create_components; use crate::shared::create_components::create_sink_writer; use crate::shared::metrics::start_metrics_server; use crate::tracker::TrackerHandle; +use crate::watermark::isb::ISBWatermarkHandle; +use crate::watermark::source::SourceWatermarkHandle; +use crate::watermark::WatermarkHandle; use crate::{error, shared, Result}; mod forwarder; @@ -29,18 +35,96 @@ pub(crate) async fn start_forwarder( cln_token: CancellationToken, config: PipelineConfig, ) -> Result<()> { + let js_context = create_js_context(config.js_client_config.clone()).await?; + match &config.vertex_config { pipeline::VertexType::Source(source) => { info!("Starting source forwarder"); - start_source_forwarder(cln_token, config.clone(), source.clone()).await?; + + // create watermark handle, if watermark is enabled + let source_watermark_handle = match &config.watermark_config { + Some(wm_config) => { + if let WatermarkConfig::Source(source_config) = wm_config { + Some(SourceWatermarkHandle::new(js_context.clone(), source_config).await?) + } else { + None + } + } + None => None, + }; + + start_source_forwarder( + cln_token, + js_context, + config.clone(), + source.clone(), + source_watermark_handle, + ) + .await?; } pipeline::VertexType::Sink(sink) => { info!("Starting sink forwarder"); - start_sink_forwarder(cln_token, config.clone(), sink.clone()).await?; + + // create watermark handle, if watermark is enabled + let edge_watermark_handle = match &config.watermark_config { + Some(wm_config) => { + if let WatermarkConfig::Edge(edge_config) = wm_config { + Some( + ISBWatermarkHandle::new( + config.vertex_name, + config.replica, + js_context.clone(), + edge_config, + ) + .await?, + ) + } else { + None + } + } + None => None, + }; + + start_sink_forwarder( + cln_token, + js_context, + config.clone(), + sink.clone(), + edge_watermark_handle, + ) + .await?; } pipeline::VertexType::Map(map) => { info!("Starting map forwarder"); - start_map_forwarder(cln_token, config.clone(), map.clone()).await?; + + // create watermark handle, if watermark is enabled + let edge_watermark_handle = match &config.watermark_config { + Some(wm_config) => { + if let WatermarkConfig::Edge(edge_config) = wm_config { + Some( + ISBWatermarkHandle::new( + config.vertex_name, + config.replica, + js_context.clone(), + edge_config, + ) + .await?, + ) + } else { + None + } + } + None => None, + }; + + start_map_forwarder( + cln_token, + js_context, + config.clone(), + map.clone(), + edge_watermark_handle, + ) + .await?; } } Ok(()) @@ -48,20 +132,22 @@ pub(crate) async fn start_forwarder( async fn start_source_forwarder( cln_token: CancellationToken, + js_context: Context, config: PipelineConfig, source_config: SourceVtxConfig, + source_watermark_handle: Option, ) -> Result<()> { - let callback_handler = config.callback_config.as_ref().map(|cb_cfg| { - CallbackHandler::new(config.vertex_name.clone(), cb_cfg.callback_concurrency) + let serving_callback_handler = config.callback_config.as_ref().map(|cb_cfg| { + CallbackHandler::new(config.vertex_name.to_string(), cb_cfg.callback_concurrency) }); - let tracker_handle = TrackerHandle::new(callback_handler); - let js_context = create_js_context(config.js_client_config.clone()).await?; + let tracker_handle = TrackerHandle::new(None, serving_callback_handler); let buffer_writer = create_buffer_writer( &config, js_context.clone(), tracker_handle.clone(), cln_token.clone(), + source_watermark_handle.clone().map(WatermarkHandle::Source), ) .await; @@ -79,6 +165,7 @@ async fn start_source_forwarder( &source_config.source_config, tracker_handle, transformer, + source_watermark_handle, cln_token.clone(), ) .await?; @@ -108,11 +195,11 @@ async fn start_source_forwarder( async fn start_map_forwarder( cln_token: CancellationToken, + js_context: Context, config: PipelineConfig, map_vtx_config: MapVtxConfig, + watermark_handle: Option, ) -> Result<()> { - let js_context = create_js_context(config.js_client_config.clone()).await?; - // Only the reader config of the first "from" vertex is needed, as all "from" vertices currently write // to a common buffer, in the case of a join. let reader_config = &config @@ -126,12 +213,13 @@ async fn start_map_forwarder( let mut mapper_grpc_client = None; let mut isb_lag_readers = vec![]; - let callback_handler = config.callback_config.as_ref().map(|cb_cfg| { - CallbackHandler::new(config.vertex_name.clone(), cb_cfg.callback_concurrency) + let serving_callback_handler = config.callback_config.as_ref().map(|cb_cfg| { + CallbackHandler::new(config.vertex_name.to_string(), cb_cfg.callback_concurrency) }); for stream in reader_config.streams.clone() { - let tracker_handle = TrackerHandle::new(callback_handler.clone()); + let tracker_handle = + TrackerHandle::new(watermark_handle.clone(), serving_callback_handler.clone()); let buffer_reader = create_buffer_reader( stream, @@ -139,6 +227,7 @@ async fn start_map_forwarder( js_context.clone(), tracker_handle.clone(), config.batch_size, + watermark_handle.clone(), ) .await?; @@ -162,6 +251,7 @@ async fn start_map_forwarder( js_context.clone(), tracker_handle.clone(), cln_token.clone(), + watermark_handle.clone().map(WatermarkHandle::ISB), ) .await; forwarder_components.push((buffer_reader, buffer_writer, mapper)); @@ -209,11 +299,11 @@ async fn start_map_forwarder( async fn start_sink_forwarder( cln_token: CancellationToken, + js_context: Context, config: PipelineConfig, sink: SinkVtxConfig, + watermark_handle: Option, ) -> Result<()> { - let js_context = create_js_context(config.js_client_config.clone()).await?; - // Only the reader config of the first "from" vertex is needed, as all "from" vertices currently write // to a common buffer, in the case of a join. let reader_config = &config @@ -222,15 +312,16 @@ async fn start_sink_forwarder( .ok_or_else(|| error::Error::Config("No from vertex config found".to_string()))? .reader_config; - let callback_handler = config.callback_config.as_ref().map(|cb_cfg| { - CallbackHandler::new(config.vertex_name.clone(), cb_cfg.callback_concurrency) + let serving_callback_handler = config.callback_config.as_ref().map(|cb_cfg| { + CallbackHandler::new(config.vertex_name.to_string(), cb_cfg.callback_concurrency) }); // Create sink writers and buffer readers for each stream let mut sink_writers = vec![]; let mut buffer_readers = vec![]; for stream in reader_config.streams.clone() { - let tracker_handle = TrackerHandle::new(callback_handler.clone()); + let tracker_handle = + TrackerHandle::new(watermark_handle.clone(), serving_callback_handler.clone()); let buffer_reader = create_buffer_reader( stream, @@ -238,6 +329,7 @@ async fn start_sink_forwarder( js_context.clone(), tracker_handle.clone(), config.batch_size, + watermark_handle.clone(), ) .await?; buffer_readers.push(buffer_reader); @@ -306,6 +398,7 @@ async fn create_buffer_writer( js_context: Context, tracker_handle: TrackerHandle, cln_token: CancellationToken, + watermark_handle: Option, ) -> JetstreamWriter { JetstreamWriter::new( config.to_vertex_config.clone(), @@ -313,23 +406,25 @@ async fn create_buffer_writer( config.paf_concurrency, tracker_handle, cln_token, + watermark_handle, ) } async fn create_buffer_reader( - stream: (&'static str, u16), + stream: Stream, reader_config: BufferReaderConfig, js_context: Context, tracker_handle: TrackerHandle, batch_size: usize, -) -> Result { - JetstreamReader::new( - stream.0, - stream.1, + watermark_handle: Option, +) -> Result { + JetStreamReader::new( + stream, js_context, reader_config, tracker_handle, batch_size, + watermark_handle, ) .await } @@ -390,11 +485,11 @@ mod tests { async fn test_forwarder_for_source_vertex() { // Unique names for the streams we use in this test let streams = vec![ - "default-test-forwarder-for-source-vertex-out-0", - "default-test-forwarder-for-source-vertex-out-1", - "default-test-forwarder-for-source-vertex-out-2", - "default-test-forwarder-for-source-vertex-out-3", - "default-test-forwarder-for-source-vertex-out-4", + Stream::new("default-test-forwarder-for-source-vertex-out-0", "test", 0), + Stream::new("default-test-forwarder-for-source-vertex-out-1", "test", 1), + Stream::new("default-test-forwarder-for-source-vertex-out-2", "test", 2), + Stream::new("default-test-forwarder-for-source-vertex-out-3", "test", 3), + Stream::new("default-test-forwarder-for-source-vertex-out-4", "test", 4), ]; let js_url = "localhost:4222"; @@ -405,14 +500,13 @@ mod tests { // Create streams to which the generator source vertex we create later will forward // messages to. The consumers created for the corresponding streams will be used to ensure // that messages were actually written to the streams. - for stream_name in &streams { - let stream_name = *stream_name; + for stream in &streams { // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.into()], max_message_size: 64 * 1024, max_messages: 10000, ..Default::default() @@ -423,20 +517,20 @@ mod tests { let c: consumer::PullConsumer = context .create_consumer_on_stream( consumer::pull::Config { - name: Some(stream_name.to_string()), + name: Some(stream.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); - consumers.push((stream_name.to_string(), c)); + consumers.push((stream.to_string(), c)); } let pipeline_config = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "in".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "in", replica: 0, batch_size: 1000, paf_concurrency: 30000, @@ -449,13 +543,9 @@ mod tests { from_vertex_config: vec![], to_vertex_config: vec![ToVertexConfig { name: "out".to_string(), + partitions: 5, writer_config: BufferWriterConfig { - streams: streams - .iter() - .enumerate() - .map(|(i, stream_name)| ((*stream_name).to_string(), i as u16)) - .collect(), - partitions: 5, + streams: streams.clone(), max_length: 30000, usage_limit: 0.8, buffer_full_strategy: RetryUntilSuccess, @@ -483,6 +573,7 @@ mod tests { lag_refresh_interval_in_secs: 3, lookback_window_in_secs: 120, }, + watermark_config: None, callback_config: None, }; @@ -520,8 +611,8 @@ mod tests { } // Delete all streams created in this test - for stream_name in streams { - context.delete_stream(stream_name).await.unwrap(); + for stream in streams { + context.delete_stream(stream.name).await.unwrap(); } } @@ -532,11 +623,11 @@ mod tests { async fn test_forwarder_for_sink_vertex() { // Unique names for the streams we use in this test let streams = vec![ - "default-test-forwarder-for-sink-vertex-out-0", - "default-test-forwarder-for-sink-vertex-out-1", - "default-test-forwarder-for-sink-vertex-out-2", - "default-test-forwarder-for-sink-vertex-out-3", - "default-test-forwarder-for-sink-vertex-out-4", + Stream::new("default-test-forwarder-for-sink-vertex-out-0", "test", 0), + Stream::new("default-test-forwarder-for-sink-vertex-out-1", "test", 1), + Stream::new("default-test-forwarder-for-sink-vertex-out-2", "test", 2), + Stream::new("default-test-forwarder-for-sink-vertex-out-3", "test", 3), + Stream::new("default-test-forwarder-for-sink-vertex-out-4", "test", 4), ]; let js_url = "localhost:4222"; @@ -545,14 +636,13 @@ mod tests { const MESSAGE_COUNT: usize = 10; let mut consumers = vec![]; - for stream_name in &streams { - let stream_name = *stream_name; + for stream in &streams { // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.into(), + subjects: vec![stream.name.into()], max_message_size: 64 * 1024, max_messages: 10000, ..Default::default() @@ -568,8 +658,9 @@ mod tests { keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), - offset: Some(Offset::String(StringOffset::new("123".to_string(), 0))), + offset: Offset::String(StringOffset::new("123".to_string(), 0)), event_time: Utc.timestamp_opt(1627846261, 0).unwrap(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "123".to_string().into(), @@ -582,7 +673,7 @@ mod tests { for _ in 0..MESSAGE_COUNT { context - .publish(stream_name.to_string(), message.clone().into()) + .publish(stream.name, message.clone().into()) .await .unwrap() .await @@ -592,20 +683,20 @@ mod tests { let c: consumer::PullConsumer = context .create_consumer_on_stream( consumer::pull::Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); - consumers.push((stream_name.to_string(), c)); + consumers.push((stream.name.to_string(), c)); } let pipeline_config = PipelineConfig { - pipeline_name: "simple-pipeline".to_string(), - vertex_name: "in".to_string(), + pipeline_name: "simple-pipeline", + vertex_name: "in", replica: 0, batch_size: 1000, paf_concurrency: 1000, @@ -619,12 +710,7 @@ mod tests { from_vertex_config: vec![FromVertexConfig { name: "in".to_string(), reader_config: BufferReaderConfig { - partitions: 5, - streams: streams - .iter() - .enumerate() - .map(|(i, key)| (*key, i as u16)) - .collect(), + streams: streams.clone(), wip_ack_interval: Duration::from_secs(1), }, partitions: 0, @@ -642,6 +728,7 @@ mod tests { lag_refresh_interval_in_secs: 3, lookback_window_in_secs: 120, }, + watermark_config: None, callback_config: None, }; @@ -679,8 +766,8 @@ mod tests { } // Delete all streams created in this test - for stream_name in streams { - context.delete_stream(stream_name).await.unwrap(); + for stream in streams { + context.delete_stream(stream.name).await.unwrap(); } } @@ -721,19 +808,19 @@ mod tests { // Unique names for the streams we use in this test let input_streams = vec![ - "default-test-forwarder-for-map-vertex-in-0", - "default-test-forwarder-for-map-vertex-in-1", - "default-test-forwarder-for-map-vertex-in-2", - "default-test-forwarder-for-map-vertex-in-3", - "default-test-forwarder-for-map-vertex-in-4", + Stream::new("default-test-forwarder-for-map-vertex-in-0", "test", 0), + Stream::new("default-test-forwarder-for-map-vertex-in-1", "test", 1), + Stream::new("default-test-forwarder-for-map-vertex-in-2", "test", 2), + Stream::new("default-test-forwarder-for-map-vertex-in-3", "test", 3), + Stream::new("default-test-forwarder-for-map-vertex-in-4", "test", 4), ]; let output_streams = vec![ - "default-test-forwarder-for-map-vertex-out-0", - "default-test-forwarder-for-map-vertex-out-1", - "default-test-forwarder-for-map-vertex-out-2", - "default-test-forwarder-for-map-vertex-out-3", - "default-test-forwarder-for-map-vertex-out-4", + Stream::new("default-test-forwarder-for-map-vertex-out-0", "test", 0), + Stream::new("default-test-forwarder-for-map-vertex-out-1", "test", 1), + Stream::new("default-test-forwarder-for-map-vertex-out-2", "test", 2), + Stream::new("default-test-forwarder-for-map-vertex-out-3", "test", 3), + Stream::new("default-test-forwarder-for-map-vertex-out-4", "test", 4), ]; let js_url = "localhost:4222"; @@ -743,14 +830,13 @@ mod tests { const MESSAGE_COUNT: usize = 10; let mut input_consumers = vec![]; let mut output_consumers = vec![]; - for stream_name in &input_streams { - let stream_name = *stream_name; + for stream in &input_streams { // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], max_message_size: 64 * 1024, max_messages: 10000, ..Default::default() @@ -766,8 +852,9 @@ mod tests { keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), - offset: Some(Offset::String(StringOffset::new("123".to_string(), 0))), + offset: Offset::String(StringOffset::new("123".to_string(), 0)), event_time: Utc.timestamp_opt(1627846261, 0).unwrap(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "123".to_string().into(), @@ -780,7 +867,7 @@ mod tests { for _ in 0..MESSAGE_COUNT { context - .publish(stream_name.to_string(), message.clone().into()) + .publish(stream.name, message.clone().into()) .await .unwrap() .await @@ -790,27 +877,26 @@ mod tests { let c: consumer::PullConsumer = context .create_consumer_on_stream( consumer::pull::Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); - input_consumers.push((stream_name.to_string(), c)); + input_consumers.push((stream.name.to_string(), c)); } // Create output streams and consumers - for stream_name in &output_streams { - let stream_name = *stream_name; + for stream in &output_streams { // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.into()], max_message_size: 64 * 1024, max_messages: 1000, ..Default::default() @@ -821,20 +907,20 @@ mod tests { let c: consumer::PullConsumer = context .create_consumer_on_stream( consumer::pull::Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); - output_consumers.push((stream_name.to_string(), c)); + output_consumers.push((stream.name.to_string(), c)); } let pipeline_config = PipelineConfig { - pipeline_name: "simple-map-pipeline".to_string(), - vertex_name: "in".to_string(), + pipeline_name: "simple-map-pipeline", + vertex_name: "in", replica: 0, batch_size: 1000, paf_concurrency: 1000, @@ -846,13 +932,9 @@ mod tests { }, to_vertex_config: vec![ToVertexConfig { name: "map-out".to_string(), + partitions: 5, writer_config: BufferWriterConfig { - streams: output_streams - .iter() - .enumerate() - .map(|(i, stream_name)| ((*stream_name).to_string(), i as u16)) - .collect(), - partitions: 5, + streams: output_streams.clone(), max_length: 30000, usage_limit: 0.8, buffer_full_strategy: RetryUntilSuccess, @@ -862,12 +944,7 @@ mod tests { from_vertex_config: vec![FromVertexConfig { name: "map-in".to_string(), reader_config: BufferReaderConfig { - partitions: 5, - streams: input_streams - .iter() - .enumerate() - .map(|(i, key)| (*key, i as u16)) - .collect(), + streams: input_streams.clone(), wip_ack_interval: Duration::from_secs(1), }, partitions: 0, @@ -887,6 +964,7 @@ mod tests { lag_refresh_interval_in_secs: 3, lookback_window_in_secs: 120, }, + watermark_config: None, callback_config: None, }; @@ -924,8 +1002,8 @@ mod tests { } // Delete all streams created in this test - for stream_name in input_streams.iter().chain(output_streams.iter()) { - context.delete_stream(stream_name).await.unwrap(); + for stream in input_streams.iter().chain(output_streams.iter()) { + context.delete_stream(stream.name).await.unwrap(); } } } diff --git a/rust/numaflow-core/src/pipeline/forwarder.rs b/rust/numaflow-core/src/pipeline/forwarder.rs index 3fb39e5a7e..86ce982b33 100644 --- a/rust/numaflow-core/src/pipeline/forwarder.rs +++ b/rust/numaflow-core/src/pipeline/forwarder.rs @@ -15,13 +15,18 @@ //! * - optional //! ``` //! -//! Most of the data move forward except for the `ack` which can happen only after the that the tracker -//! has guaranteed that the processing complete. +//! Most of the data move forward except for the `ack`, `watermark` which can happen only after the +//! that the tracker has guaranteed that the processing complete. Ack is spawned during the reading. //! ```text //! (Read) +-------> (UDF) -------> (Write) + //! | | //! | | -//! +-------> {Ack} <----------------+ +//! +-------> {tracker} <------------ +//! | +//! +-----------+-----------+ +//! | | | +//! v v v +//! (track watermark) (callbacks) {ack} //! //! {} -> Listens on a OneShot //! () -> Streaming Interface diff --git a/rust/numaflow-core/src/pipeline/forwarder/map_forwarder.rs b/rust/numaflow-core/src/pipeline/forwarder/map_forwarder.rs index afc08a6672..fb7ae959af 100644 --- a/rust/numaflow-core/src/pipeline/forwarder/map_forwarder.rs +++ b/rust/numaflow-core/src/pipeline/forwarder/map_forwarder.rs @@ -2,14 +2,14 @@ use tokio_util::sync::CancellationToken; use crate::error::Error; use crate::mapper::map::MapHandle; -use crate::pipeline::isb::jetstream::reader::JetstreamReader; +use crate::pipeline::isb::jetstream::reader::JetStreamReader; use crate::pipeline::isb::jetstream::writer::JetstreamWriter; use crate::Result; /// Map forwarder is a component which starts a streaming reader, a mapper, and a writer /// and manages the lifecycle of these components. pub(crate) struct MapForwarder { - jetstream_reader: JetstreamReader, + jetstream_reader: JetStreamReader, mapper: MapHandle, jetstream_writer: JetstreamWriter, cln_token: CancellationToken, @@ -17,7 +17,7 @@ pub(crate) struct MapForwarder { impl MapForwarder { pub(crate) async fn new( - jetstream_reader: JetstreamReader, + jetstream_reader: JetStreamReader, mapper: MapHandle, jetstream_writer: JetstreamWriter, cln_token: CancellationToken, diff --git a/rust/numaflow-core/src/pipeline/forwarder/sink_forwarder.rs b/rust/numaflow-core/src/pipeline/forwarder/sink_forwarder.rs index 1d560e94e5..c73d58ec09 100644 --- a/rust/numaflow-core/src/pipeline/forwarder/sink_forwarder.rs +++ b/rust/numaflow-core/src/pipeline/forwarder/sink_forwarder.rs @@ -1,21 +1,21 @@ use tokio_util::sync::CancellationToken; use crate::error::Error; -use crate::pipeline::isb::jetstream::reader::JetstreamReader; +use crate::pipeline::isb::jetstream::reader::JetStreamReader; use crate::sink::SinkWriter; use crate::Result; /// Sink forwarder is a component which starts a streaming reader and a sink writer /// and manages the lifecycle of these components. pub(crate) struct SinkForwarder { - jetstream_reader: JetstreamReader, + jetstream_reader: JetStreamReader, sink_writer: SinkWriter, cln_token: CancellationToken, } impl SinkForwarder { pub(crate) async fn new( - jetstream_reader: JetstreamReader, + jetstream_reader: JetStreamReader, sink_writer: SinkWriter, cln_token: CancellationToken, ) -> Self { diff --git a/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs b/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs index a14ab7b4c8..c7128482dd 100644 --- a/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs +++ b/rust/numaflow-core/src/pipeline/forwarder/source_forwarder.rs @@ -68,7 +68,7 @@ mod tests { use tokio::task::JoinHandle; use tokio_util::sync::CancellationToken; - use crate::config::pipeline::isb::BufferWriterConfig; + use crate::config::pipeline::isb::{BufferWriterConfig, Stream}; use crate::config::pipeline::ToVertexConfig; use crate::pipeline::forwarder::source_forwarder::SourceForwarder; use crate::pipeline::isb::jetstream::writer::JetstreamWriter; @@ -163,7 +163,7 @@ mod tests { #[cfg(feature = "nats-tests")] #[tokio::test] async fn test_source_forwarder() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); // create the source which produces x number of messages let cln_token = CancellationToken::new(); @@ -226,6 +226,7 @@ mod tests { tracker_handle.clone(), true, Some(transformer), + None, ); // create a js writer @@ -234,13 +235,13 @@ mod tests { let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_source_forwarder"; + let stream = Stream::new("test_source_forwarder", "test", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.into()], max_message_size: 1024, ..Default::default() }) @@ -250,19 +251,20 @@ mod tests { let _consumer = context .create_consumer_on_stream( consumer::Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); let writer = JetstreamWriter::new( vec![ToVertexConfig { + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], ..Default::default() }, conditions: None, @@ -272,6 +274,7 @@ mod tests { 100, tracker_handle.clone(), cln_token.clone(), + None, ); // create the forwarder with the source, transformer, and writer diff --git a/rust/numaflow-core/src/pipeline/isb.rs b/rust/numaflow-core/src/pipeline/isb.rs index e59d0b983e..02e3e45970 100644 --- a/rust/numaflow-core/src/pipeline/isb.rs +++ b/rust/numaflow-core/src/pipeline/isb.rs @@ -1,3 +1,8 @@ +//! [Vertex] are connected via [ISB]. +//! +//! [Vertex]: https://numaflow.numaproj.io/core-concepts/vertex/ +//! [ISB]: https://numaflow.numaproj.io/core-concepts/inter-step-buffer/ + // TODO: implement a simple ISB and a trait for ISB pub(crate) mod jetstream; diff --git a/rust/numaflow-core/src/pipeline/isb/jetstream.rs b/rust/numaflow-core/src/pipeline/isb/jetstream.rs index c7a8f74c1e..f348d0275c 100644 --- a/rust/numaflow-core/src/pipeline/isb/jetstream.rs +++ b/rust/numaflow-core/src/pipeline/isb/jetstream.rs @@ -6,6 +6,3 @@ pub(crate) mod writer; pub(crate) mod reader; - -/// Stream is a combination of stream name and partition id. -type Stream = (String, u16); diff --git a/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs b/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs index 97aa5f9864..ffbe3fc8f2 100644 --- a/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs +++ b/rust/numaflow-core/src/pipeline/isb/jetstream/reader.rs @@ -1,9 +1,11 @@ use std::fmt; +use std::sync::Arc; use std::time::Duration; use async_nats::jetstream::{ consumer::PullConsumer, AckKind, Context, Message as JetstreamMessage, }; +use prost::Message as ProtoMessage; use tokio::sync::{mpsc, oneshot}; use tokio::task::JoinHandle; use tokio::time::{self, Instant}; @@ -13,42 +15,104 @@ use tokio_util::sync::CancellationToken; use tracing::{error, info}; use crate::config::get_vertex_name; -use crate::config::pipeline::isb::BufferReaderConfig; +use crate::config::pipeline::isb::{BufferReaderConfig, Stream}; use crate::error::Error; -use crate::message::{IntOffset, Message, MessageID, Offset, ReadAck}; +use crate::message::{IntOffset, Message, MessageID, Metadata, Offset, ReadAck}; use crate::metrics::{ pipeline_forward_metric_labels, pipeline_isb_metric_labels, pipeline_metrics, }; +use crate::shared::grpc::utc_from_timestamp; use crate::tracker::TrackerHandle; +use crate::watermark::isb::ISBWatermarkHandle; use crate::Result; -/// The JetstreamReader is a handle to the background actor that continuously fetches messages from Jetstream. -/// It can be used to cancel the background task and stop reading from Jetstream. +/// The JetStreamReader is a handle to the background actor that continuously fetches messages from JetStream. +/// It can be used to cancel the background task and stop reading from JetStream. /// The sender end of the channel is not stored in this struct, since the struct is clone-able and the mpsc channel is only closed when all the senders are dropped. /// Storing the Sender end of channel in this struct would make it difficult to close the channel with `cancel` method. #[derive(Clone)] -pub(crate) struct JetstreamReader { - stream_name: &'static str, - partition_idx: u16, +pub(crate) struct JetStreamReader { + stream: Stream, config: BufferReaderConfig, consumer: PullConsumer, tracker_handle: TrackerHandle, batch_size: usize, + watermark_handle: Option, +} + +/// JSWrappedMessage is a wrapper around the JetStream message that includes the +/// partition index and the vertex name. +struct JSWrappedMessage { + partition_idx: u16, + message: async_nats::jetstream::Message, + vertex_name: String, +} + +impl TryFrom for Message { + type Error = Error; + + fn try_from(value: JSWrappedMessage) -> Result { + let msg_info = value.message.info().map_err(|e| { + Error::ISB(format!( + "Failed to get message info from JetStream: {:?}", + e + )) + })?; + + let proto_message = + numaflow_pb::objects::isb::Message::decode(value.message.payload.clone()) + .map_err(|e| Error::Proto(e.to_string()))?; + + let header = proto_message + .header + .ok_or(Error::Proto("Missing header".to_string()))?; + let body = proto_message + .body + .ok_or(Error::Proto("Missing body".to_string()))?; + let message_info = header + .message_info + .ok_or(Error::Proto("Missing message_info".to_string()))?; + let offset = Offset::Int(IntOffset::new( + msg_info.stream_sequence as i64, + value.partition_idx, + )); + + Ok(Message { + keys: Arc::from(header.keys.into_boxed_slice()), + tags: None, + value: body.payload.into(), + offset: offset.clone(), + event_time: utc_from_timestamp(message_info.event_time), + id: MessageID { + vertex_name: value.vertex_name.into(), + offset: offset.to_string().into(), + index: 0, + }, + headers: header.headers, + watermark: None, + metadata: Some(Metadata { + previous_vertex: header + .id + .ok_or(Error::Proto("Missing id".to_string()))? + .vertex_name, + }), + }) + } } -impl JetstreamReader { +impl JetStreamReader { pub(crate) async fn new( - stream_name: &'static str, - partition_idx: u16, + stream: Stream, js_ctx: Context, config: BufferReaderConfig, tracker_handle: TrackerHandle, batch_size: usize, + watermark_handle: Option, ) -> Result { let mut config = config; let mut consumer: PullConsumer = js_ctx - .get_consumer_from_stream(&stream_name, &stream_name) + .get_consumer_from_stream(&stream.name, &stream.name) .await .map_err(|e| Error::ISB(format!("Failed to get consumer for stream {}", e)))?; @@ -66,16 +130,16 @@ impl JetstreamReader { config.wip_ack_interval = wip_ack_interval; Ok(Self { - stream_name, - partition_idx, + stream, config: config.clone(), consumer, tracker_handle, batch_size, + watermark_handle, }) } - /// streaming_read is a background task that continuously fetches messages from Jetstream and + /// streaming_read is a background task that continuously fetches messages from JetStream and /// emits them on a channel. When we encounter an error, we log the error and return from the /// function. This drops the sender end of the channel. The closing of the channel should propagate /// to the receiver end and the receiver should exit gracefully. Within the loop, we only consider @@ -89,14 +153,14 @@ impl JetstreamReader { let handle: JoinHandle> = tokio::spawn({ let consumer = self.consumer.clone(); - let partition_idx = self.partition_idx; + let stream = self.stream.clone(); let config = self.config.clone(); let tracker_handle = self.tracker_handle.clone(); let cancel_token = cancel_token.clone(); + let watermark_handle = self.watermark_handle.clone(); - let stream_name = self.stream_name; async move { - let labels = pipeline_forward_metric_labels("Sink", Some(stream_name)); + let labels = pipeline_forward_metric_labels("Sink", Some(stream.name)); let mut message_stream = consumer.messages().await.map_err(|e| { Error::ISB(format!( @@ -108,61 +172,45 @@ impl JetstreamReader { loop { tokio::select! { _ = cancel_token.cancelled() => { // should we drain from the stream when token is cancelled? - info!(?stream_name, "Cancellation token received, stopping the reader."); + info!(?stream, "Cancellation token received, stopping the reader."); break; } message = message_stream.next() => { let Some(message) = message else { // stream has been closed because we got none - info!(?stream_name, "Stream has been closed"); + info!(?stream, "Stream has been closed"); break; }; let jetstream_message = match message { Ok(message) => message, Err(e) => { - error!(?e, ?stream_name, "Failed to fetch messages from the Jetstream"); + error!(?e, ?stream, "Failed to fetch messages from the Jetstream"); continue; } }; - let msg_info = match jetstream_message.info() { - Ok(info) => info, - Err(e) => { - error!(?e, ?stream_name, "Failed to get message info from Jetstream"); - continue; - } + + let js_message = JSWrappedMessage { + partition_idx: stream.partition, + message: jetstream_message.clone(), + vertex_name: get_vertex_name().to_string(), }; - let mut message: Message = match jetstream_message.payload.clone().try_into() { + let mut message: Message = match js_message.try_into() { Ok(message) => message, Err(e) => { error!( - ?e, ?stream_name, ?jetstream_message, + ?e, ?stream, ?jetstream_message, "Failed to parse message payload received from Jetstream", ); continue; } }; - let offset = Offset::Int(IntOffset::new( - msg_info.stream_sequence, - partition_idx, - )); - - let message_id = MessageID { - vertex_name: get_vertex_name().to_string().into(), - offset: offset.to_string().into(), - index: 0, - }; - - let metadata = crate::message::Metadata{ - // Copy previous vertex name from message id - previous_vertex: String::from_utf8_lossy(&message.id.vertex_name).into(), - }; - message.metadata = Some(metadata); - - message.offset = Some(offset.clone()); - message.id = message_id.clone(); + if let Some(watermark_handle) = watermark_handle.as_ref() { + let watermark = watermark_handle.fetch_watermark(message.offset.clone()).await?; + message.watermark = Some(watermark); + } // Insert the message into the tracker and wait for the ack to be sent back. let (ack_tx, ack_rx) = oneshot::channel(); @@ -174,9 +222,10 @@ impl JetstreamReader { config.wip_ack_interval, )); + let offset = message.offset.clone(); if let Err(e) = messages_tx.send(message).await { // nak the read message and return - tracker_handle.discard(message_id.offset.clone()).await?; + tracker_handle.discard(offset).await?; return Err(Error::ISB(format!( "Failed to send message to receiver: {:?}", e @@ -263,23 +312,23 @@ impl JetstreamReader { let x = self.consumer.info().await.map_err(|e| { Error::ISB(format!( "Failed to get consumer info for stream {}: {}", - self.stream_name, e + self.stream.name, e )) })?; Ok(Some(x.num_pending as usize + x.num_ack_pending)) } pub(crate) fn name(&self) -> &'static str { - self.stream_name + self.stream.name } } -impl fmt::Display for JetstreamReader { +impl fmt::Display for JetStreamReader { fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { write!( f, "JetstreamReader {{ stream_name: {}, partition_idx: {}, config: {:?} }}", - self.stream_name, self.partition_idx, self.config + self.stream, self.stream.partition, self.config ) } } @@ -306,13 +355,13 @@ mod tests { let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_jetstream_read"; + let stream = Stream::new("test_jetstream_read", "test", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], max_message_size: 1024, ..Default::default() }) @@ -322,27 +371,26 @@ mod tests { let _consumer = context .create_consumer_on_stream( consumer::Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); let buf_reader_config = BufferReaderConfig { - partitions: 0, streams: vec![], wip_ack_interval: Duration::from_millis(5), }; - let js_reader = JetstreamReader::new( - stream_name, - 0, + let js_reader = JetStreamReader::new( + stream.clone(), context.clone(), buf_reader_config, - TrackerHandle::new(None), + TrackerHandle::new(None, None), 500, + None, ) .await .unwrap(); @@ -358,19 +406,20 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, }; let message_bytes: BytesMut = message.try_into().unwrap(); context - .publish(stream_name, message_bytes.into()) + .publish(stream.name, message_bytes.into()) .await .unwrap(); } @@ -386,13 +435,13 @@ mod tests { assert_eq!( buffer.len(), 10, - "Expected 10 messages from the Jestream reader" + "Expected 10 messages from the jetstream reader" ); reader_cancel_token.cancel(); js_reader_task.await.unwrap().unwrap(); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -402,15 +451,15 @@ mod tests { // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); - let stream_name = "test_ack"; + let js_stream = Stream::new("test-ack", "test", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(js_stream.name).await; context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: js_stream.to_string(), + subjects: vec![js_stream.to_string()], max_message_size: 1024, ..Default::default() }) @@ -420,27 +469,26 @@ mod tests { let _consumer = context .create_consumer_on_stream( consumer::Config { - name: Some(stream_name.to_string()), + name: Some(js_stream.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + js_stream.name.to_string(), ) .await .unwrap(); let buf_reader_config = BufferReaderConfig { - partitions: 0, streams: vec![], wip_ack_interval: Duration::from_millis(5), }; - let js_reader = JetstreamReader::new( - stream_name, - 0, + let js_reader = JetStreamReader::new( + js_stream.clone(), context.clone(), buf_reader_config, tracker_handle.clone(), 1, + None, ) .await .unwrap(); @@ -458,20 +506,21 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i + 1, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("{}-0", i + 1).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, }; - offsets.push(message.id.offset.clone()); + offsets.push(message.offset.clone()); let message_bytes: BytesMut = message.try_into().unwrap(); context - .publish(stream_name, message_bytes.into()) + .publish(js_stream.name, message_bytes.into()) .await .unwrap(); } @@ -497,7 +546,7 @@ mod tests { .expect("Tracker is not empty after 1 second"); let mut consumer: PullConsumer = context - .get_consumer_from_stream(stream_name, stream_name) + .get_consumer_from_stream(js_stream.name, js_stream.name) .await .unwrap(); @@ -509,6 +558,6 @@ mod tests { reader_cancel_token.cancel(); js_reader_task.await.unwrap().unwrap(); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(js_stream.name).await.unwrap(); } } diff --git a/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs b/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs index 063af6391a..0521d25a32 100644 --- a/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs +++ b/rust/numaflow-core/src/pipeline/isb/jetstream/writer.rs @@ -12,21 +12,20 @@ use async_nats::jetstream::Context; use bytes::{Bytes, BytesMut}; use tokio::sync::Semaphore; use tokio::task::JoinHandle; -use tokio::time; use tokio::time::{sleep, Instant}; use tokio_stream::wrappers::ReceiverStream; use tokio_stream::StreamExt; use tokio_util::sync::CancellationToken; use tracing::{debug, error, info, warn}; -use crate::config::pipeline::isb::BufferFullStrategy; +use crate::config::pipeline::isb::{BufferFullStrategy, Stream}; use crate::config::pipeline::ToVertexConfig; use crate::error::Error; use crate::message::{IntOffset, Message, Offset}; use crate::metrics::{pipeline_isb_metric_labels, pipeline_metrics}; -use crate::pipeline::isb::jetstream::Stream; use crate::shared::forward; use crate::tracker::TrackerHandle; +use crate::watermark::WatermarkHandle; use crate::Result; const DEFAULT_RETRY_INTERVAL_MILLIS: u64 = 10; @@ -40,10 +39,13 @@ const DEFAULT_REFRESH_INTERVAL_SECS: u64 = 1; pub(crate) struct JetstreamWriter { config: Arc>, js_ctx: Context, - is_full: HashMap>, + /// HashMap of streams (a vertex can write to any immediate downstream) and a bool to represent + /// whether the corresponding stream is full. + is_full: HashMap<&'static str, Arc>, cancel_token: CancellationToken, tracker_handle: TrackerHandle, sem: Arc, + watermark_handle: Option, } impl JetstreamWriter { @@ -55,6 +57,7 @@ impl JetstreamWriter { paf_concurrency: usize, tracker_handle: TrackerHandle, cancel_token: CancellationToken, + watermark_handle: Option, ) -> Self { let streams = config .iter() @@ -63,7 +66,7 @@ impl JetstreamWriter { let is_full = streams .iter() - .map(|stream| (stream.0.clone(), Arc::new(AtomicBool::new(false)))) + .map(|stream| (stream.name, Arc::new(AtomicBool::new(false)))) .collect::>(); let this = Self { @@ -73,6 +76,7 @@ impl JetstreamWriter { cancel_token, tracker_handle, sem: Arc::new(Semaphore::new(paf_concurrency)), + watermark_handle, }; // spawn a task for checking whether buffer is_full @@ -96,19 +100,20 @@ impl JetstreamWriter { _ = interval.tick() => { for config in &*self.config { for stream in &config.writer_config.streams { - match Self::fetch_buffer_usage(self.js_ctx.clone(), stream.0.as_str(), config.writer_config.max_length).await { + let stream = stream.name; + match Self::fetch_buffer_usage(self.js_ctx.clone(), stream, config.writer_config.max_length).await { Ok((soft_usage, solid_usage)) => { if solid_usage >= config.writer_config.usage_limit && soft_usage >= config.writer_config.usage_limit { - if let Some(is_full) = self.is_full.get(stream.0.as_str()) { + if let Some(is_full) = self.is_full.get(stream) { is_full.store(true, Ordering::Relaxed); } - } else if let Some(is_full) = self.is_full.get(stream.0.as_str()) { + } else if let Some(is_full) = self.is_full.get(stream) { is_full.store(false, Ordering::Relaxed); } } Err(e) => { - error!(?e, "Failed to fetch buffer usage for stream {}, updating isFull to true", stream.0.as_str()); - if let Some(is_full) = self.is_full.get(stream.0.as_str()) { + error!(?e, "Failed to fetch buffer usage for stream {}, updating isFull to true", stream); + if let Some(is_full) = self.is_full.get(stream) { is_full.store(true, Ordering::Relaxed); } } @@ -181,19 +186,19 @@ impl JetstreamWriter { let this = self.clone(); let handle: JoinHandle> = tokio::spawn(async move { - tracing::info!("Starting streaming Jetstream writer"); + info!("Starting streaming Jetstream writer"); let mut messages_stream = messages_stream; let mut hash = DefaultHasher::new(); let mut processed_msgs_count: usize = 0; - let mut last_logged_at = time::Instant::now(); + let mut last_logged_at = Instant::now(); while let Some(message) = messages_stream.next().await { // if message needs to be dropped, ack and continue // TODO: add metric for dropped count if message.dropped() { // delete the entry from tracker - this.tracker_handle.delete(message.id.offset).await?; + this.tracker_handle.delete(message.offset).await?; continue; } @@ -208,7 +213,7 @@ impl JetstreamWriter { // check to which partition the message should be written let partition = forward::determine_partition( String::from_utf8_lossy(&message.id.offset).to_string(), - vertex.writer_config.partitions, + vertex.partitions, &mut hash, ); @@ -254,7 +259,7 @@ impl JetstreamWriter { last_logged_at = Instant::now(); } } - tracing::info!("Streaming jetstream writer finished"); + info!("Streaming jetstream writer finished"); Ok(()) }); Ok(handle) @@ -271,7 +276,7 @@ impl JetstreamWriter { ) -> Option { let mut counter = 500u16; - let offset = message.id.offset.clone(); + let offset = message.offset.clone(); let payload: BytesMut = message .try_into() .expect("message serialization should not fail"); @@ -281,13 +286,13 @@ impl JetstreamWriter { // let's write only if the buffer is not full for the stream match self .is_full - .get(&stream.0) + .get(stream.name) .map(|is_full| is_full.load(Ordering::Relaxed)) { Some(true) => { // FIXME: add metrics if counter >= 500 { - warn!(stream=?stream.0, "stream is full (throttled logging)"); + warn!(stream=?stream, "stream is full (throttled logging)"); counter = 0; } counter += 1; @@ -305,7 +310,7 @@ impl JetstreamWriter { } Some(false) => match self .js_ctx - .publish(stream.0.clone(), Bytes::from(payload.clone())) + .publish(stream.name, Bytes::from(payload.clone())) .await { Ok(paf) => { @@ -316,7 +321,7 @@ impl JetstreamWriter { } }, None => { - error!("Stream {} not found in is_full map", stream.0); + error!("Stream {} not found in is_full map", stream); } } // short-circuit out in failure mode if shutdown has been initiated @@ -336,7 +341,7 @@ impl JetstreamWriter { /// natural backpressure. pub(super) async fn resolve_pafs( &self, - pafs: Vec<((String, u16), PublishAckFuture)>, + pafs: Vec<(Stream, PublishAckFuture)>, message: Message, ) -> Result<()> { let start_time = Instant::now(); @@ -345,28 +350,31 @@ impl JetstreamWriter { .await .map_err(|_e| Error::ISB("Failed to acquire semaphore permit".to_string()))?; - let mut offsets = Vec::new(); let js_ctx = self.js_ctx.clone(); let cancel_token = self.cancel_token.clone(); let tracker_handle = self.tracker_handle.clone(); + let watermark_handle = self.watermark_handle.clone(); tokio::spawn(async move { let _permit = permit; + let mut offsets = Vec::new(); + + // resolve the pafs for (stream, paf) in pafs { match paf.await { Ok(ack) => { if ack.duplicate { warn!( "Duplicate message detected for stream {}, ignoring {:?}", - stream.0, ack + stream, ack ); } offsets.push(( stream.clone(), - Offset::Int(IntOffset::new(ack.sequence, stream.1)), + Offset::Int(IntOffset::new(ack.sequence as i64, stream.partition)), )); tracker_handle - .delete(message.id.offset.clone()) + .delete(message.offset.clone()) .await .expect("Failed to delete offset from tracker"); } @@ -374,7 +382,7 @@ impl JetstreamWriter { error!( ?e, "Failed to resolve the future for stream {}, trying blocking write", - stream.0 + stream ); match JetstreamWriter::blocking_write( stream.clone(), @@ -388,19 +396,26 @@ impl JetstreamWriter { if ack.duplicate { warn!( "Duplicate message detected for stream {}, ignoring {:?}", - stream.0, ack + stream, ack ); } offsets.push(( stream.clone(), - Offset::Int(IntOffset::new(ack.sequence, stream.1)), + Offset::Int(IntOffset::new( + ack.sequence as i64, + stream.partition, + )), )); + tracker_handle + .delete(message.offset.clone()) + .await + .expect("Failed to delete offset from tracker"); } Err(e) => { - error!(?e, "Blocking write failed for stream {}", stream.0); + error!(?e, "Blocking write failed for stream {}", stream); // Since we failed to write to the stream, we need to send a NAK to the reader tracker_handle - .discard(message.id.offset.clone()) + .discard(message.offset.clone()) .await .expect("Failed to discard offset from the tracker"); return; @@ -409,12 +424,44 @@ impl JetstreamWriter { } } } + + // now the pafs have resolved, lets use the offsets to send watermark + for (stream, offset) in offsets { + if let Some(watermark_handle) = watermark_handle.as_ref() { + match watermark_handle { + WatermarkHandle::ISB(handle) => { + handle + .publish_watermark(stream, offset) + .await + .map_err(|e| { + Error::ISB(format!("Failed to update watermark: {:?}", e)) + }) + .expect("Failed to publish watermark"); + } + WatermarkHandle::Source(handle) => { + let input_partition = match &message.offset { + Offset::Int(offset) => offset.partition_idx, + Offset::String(offset) => offset.partition_idx, + }; + handle + .publish_source_isb_watermark(stream, offset, input_partition) + .await + .map_err(|e| { + Error::ISB(format!("Failed to update watermark: {:?}", e)) + }) + .expect("Failed to publish watermark"); + } + } + } + } + pipeline_metrics() .isb .paf_resolution_time .get_or_create(pipeline_isb_metric_labels()) .observe(start_time.elapsed().as_micros() as f64); }); + Ok(()) } @@ -428,9 +475,9 @@ impl JetstreamWriter { cln_token: CancellationToken, ) -> Result { let start_time = Instant::now(); - info!("Blocking write for stream {}", stream.0); + info!("Blocking write for stream {}", stream); loop { - match js_ctx.publish(stream.0.clone(), payload.clone()).await { + match js_ctx.publish(stream.name, payload.clone()).await { Ok(paf) => match paf.await { Ok(ack) => { if ack.duplicate { @@ -476,26 +523,26 @@ mod tests { use numaflow_models::models::TagConditions; use super::*; + use crate::config::pipeline::isb::BufferWriterConfig; use crate::message::{Message, MessageID, ReadAck}; - use crate::pipeline::pipeline::isb::BufferWriterConfig; #[cfg(feature = "nats-tests")] #[tokio::test] async fn test_async_write() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let cln_token = CancellationToken::new(); let js_url = "localhost:4222"; // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_async"; + let stream = Stream::new("test-async", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], ..Default::default() }) .await @@ -504,11 +551,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); @@ -516,8 +563,9 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { name: "test-vertex".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], ..Default::default() }, conditions: None, @@ -526,14 +574,16 @@ mod tests { 100, tracker_handle, cln_token.clone(), + None, ); let message = Message { keys: Arc::from(vec!["key_0".to_string()]), tags: None, value: "message 0".as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "offset_0".to_string().into(), @@ -545,14 +595,14 @@ mod tests { let paf = writer .write( - (stream_name.to_string(), 0), + stream.clone(), message, BufferFullStrategy::RetryUntilSuccess, ) .await; assert!(paf.unwrap().await.is_ok()); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -564,13 +614,13 @@ mod tests { let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_sync"; + let stream = Stream::new("test-sync", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], ..Default::default() }) .await @@ -579,11 +629,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); @@ -592,8 +642,9 @@ mod tests { keys: Arc::from(vec!["key_0".to_string()]), tags: None, value: "message 0".as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "offset_0".to_string().into(), @@ -605,7 +656,7 @@ mod tests { let message_bytes: BytesMut = message.try_into().unwrap(); let result = JetstreamWriter::blocking_write( - (stream_name.to_string(), 0), + stream.clone(), message_bytes.into(), context.clone(), cln_token.clone(), @@ -614,27 +665,27 @@ mod tests { assert!(result.is_ok()); let publish_ack = result.unwrap(); - assert_eq!(publish_ack.stream, stream_name); + assert_eq!(publish_ack.stream, stream.name); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] #[tokio::test] async fn test_write_with_cancellation() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let js_url = "localhost:4222"; // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_cancellation"; + let stream = Stream::new("test-cancellation", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], max_message_size: 1024, ..Default::default() }) @@ -644,11 +695,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name.to_string(), ) .await .unwrap(); @@ -658,8 +709,9 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { name: "test-vertex".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], ..Default::default() }, conditions: None, @@ -668,6 +720,7 @@ mod tests { 100, tracker_handle, cancel_token.clone(), + None, ); let mut result_receivers = Vec::new(); @@ -677,19 +730,20 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), + watermark: None, metadata: None, }; let paf = writer .write( - (stream_name.to_string(), 0), + stream.clone(), message, BufferFullStrategy::RetryUntilSuccess, ) @@ -703,8 +757,9 @@ mod tests { keys: Arc::from(vec!["key_11".to_string()]), tags: None, value: vec![0; 1025].into(), - offset: None, + offset: Offset::Int(IntOffset::new(11, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "offset_11".to_string().into(), @@ -715,7 +770,7 @@ mod tests { }; let paf = writer .write( - (stream_name.to_string(), 0), + stream.clone(), message, BufferFullStrategy::RetryUntilSuccess, ) @@ -741,7 +796,7 @@ mod tests { } } - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -752,13 +807,13 @@ mod tests { let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_fetch_buffer_usage"; + let stream = Stream::new("test_fetch_buffer_usage", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], max_messages: 1000, max_message_size: 1024, max_messages_per_subject: 1000, @@ -771,22 +826,22 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name.to_string(), ) .await; let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name.to_string(), ) .await .unwrap(); @@ -796,14 +851,14 @@ mod tests { // Publish messages to fill the buffer for _ in 0..80 { context - .publish(stream_name, Bytes::from("test message")) + .publish(stream.name, Bytes::from("test message")) .await .unwrap(); } // Fetch buffer usage let (soft_usage, _) = - JetstreamWriter::fetch_buffer_usage(context.clone(), stream_name, max_length) + JetstreamWriter::fetch_buffer_usage(context.clone(), stream.name, max_length) .await .unwrap(); @@ -813,28 +868,28 @@ mod tests { // Clean up context - .delete_consumer_from_stream(stream_name, stream_name) + .delete_consumer_from_stream(stream.name, stream.name) .await .unwrap(); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] #[tokio::test] async fn test_check_stream_status() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let js_url = "localhost:4222"; // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let stream_name = "test_check_stream_status"; + let stream = Stream::new("test_check_stream_status", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.to_string()], max_messages: 1000, max_message_size: 1024, max_messages_per_subject: 1000, @@ -847,11 +902,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name.to_string(), ) .await .unwrap(); @@ -860,8 +915,9 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { name: "test-vertex".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], max_length: 100, ..Default::default() }, @@ -871,6 +927,7 @@ mod tests { 100, tracker_handle, cancel_token.clone(), + None, ); let mut js_writer = writer.clone(); @@ -882,7 +939,7 @@ mod tests { // Publish messages to fill the buffer, since max_length is 100, we need to publish 80 messages for _ in 0..80 { context - .publish(stream_name, Bytes::from("test message")) + .publish(stream.name, Bytes::from("test message")) .await .unwrap(); } @@ -890,7 +947,7 @@ mod tests { let start_time = Instant::now(); while !writer .is_full - .get(stream_name) + .get(stream.name) .map(|is_full| is_full.load(Ordering::Relaxed)) .unwrap() && start_time.elapsed().as_millis() < 1000 @@ -902,14 +959,14 @@ mod tests { assert!( writer .is_full - .get(stream_name) + .get(stream.name) .map(|is_full| is_full.load(Ordering::Relaxed)) .unwrap(), "Buffer should be full after publishing messages" ); // Clean up - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -920,15 +977,15 @@ mod tests { // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); - let stream_name = "test_publish_messages"; + let stream = Stream::new("test_publish_messages", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.into()], max_messages: 1000, ..Default::default() }) @@ -938,11 +995,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); @@ -950,8 +1007,9 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { name: "test-vertex".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], max_length: 1000, ..Default::default() }, @@ -961,6 +1019,7 @@ mod tests { 100, tracker_handle.clone(), cln_token.clone(), + None, ); let (messages_tx, messages_rx) = tokio::sync::mpsc::channel(500); @@ -971,12 +1030,13 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -996,7 +1056,7 @@ mod tests { } // make sure all messages are acked assert!(tracker_handle.is_empty().await.unwrap()); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -1006,15 +1066,15 @@ mod tests { // Create JetStream context let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); - let stream_name = "test_publish_cancellation"; + let stream = Stream::new("test_publish_cancellation", "temp", 0); // Delete stream if it exists - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream.name).await; let _stream = context .get_or_create_stream(stream::Config { - name: stream_name.into(), - subjects: vec![stream_name.into()], + name: stream.name.to_string(), + subjects: vec![stream.name.into()], max_message_size: 1024, ..Default::default() }) @@ -1024,11 +1084,11 @@ mod tests { let _consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream.name, ) .await .unwrap(); @@ -1037,8 +1097,9 @@ mod tests { let writer = JetstreamWriter::new( vec![ToVertexConfig { name: "test-vertex".to_string(), + partitions: 1, writer_config: BufferWriterConfig { - streams: vec![(stream_name.to_string(), 0)], + streams: vec![stream.clone()], ..Default::default() }, conditions: None, @@ -1047,6 +1108,7 @@ mod tests { 100, tracker_handle.clone(), cancel_token.clone(), + None, ); let (tx, rx) = tokio::sync::mpsc::channel(500); @@ -1057,12 +1119,13 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -1082,8 +1145,9 @@ mod tests { keys: Arc::from(vec!["key_101".to_string()]), tags: None, value: vec![0; 1025].into(), - offset: None, + offset: Offset::Int(IntOffset::new(101, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "offset_101".to_string().into(), @@ -1112,7 +1176,7 @@ mod tests { // make sure all messages are acked assert!(tracker_handle.is_empty().await.unwrap()); - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream.name).await.unwrap(); } #[cfg(feature = "nats-tests")] @@ -1121,12 +1185,21 @@ mod tests { let js_url = "localhost:4222"; let client = async_nats::connect(js_url).await.unwrap(); let context = jetstream::new(client); - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let cln_token = CancellationToken::new(); - let vertex1_streams = vec!["vertex1-0", "vertex1-1"]; - let vertex2_streams = vec!["vertex2-0", "vertex2-1"]; - let vertex3_streams = vec!["vertex3-0", "vertex3-1"]; + let vertex1_streams = vec![ + Stream::new("vertex1-0", "temp", 0), + Stream::new("vertex1-1", "temp", 0), + ]; + let vertex2_streams = vec![ + Stream::new("vertex2-0", "temp", 0), + Stream::new("vertex2-1", "temp", 0), + ]; + let vertex3_streams = vec![ + Stream::new("vertex3-0", "temp", 0), + Stream::new("vertex3-1", "temp", 0), + ]; let (_, consumers1) = create_streams_and_consumers(&context, &vertex1_streams).await; let (_, consumers2) = create_streams_and_consumers(&context, &vertex2_streams).await; @@ -1136,12 +1209,9 @@ mod tests { vec![ ToVertexConfig { name: "vertex1".to_string(), + partitions: 2, writer_config: BufferWriterConfig { - streams: vec![ - (vertex1_streams[0].to_string(), 0), - (vertex1_streams[1].to_string(), 1), - ], - partitions: 2, + streams: vertex1_streams.clone(), ..Default::default() }, conditions: Some(Box::new(ForwardConditions::new(TagConditions { @@ -1151,12 +1221,9 @@ mod tests { }, ToVertexConfig { name: "vertex2".to_string(), + partitions: 2, writer_config: BufferWriterConfig { - streams: vec![ - (vertex2_streams[0].to_string(), 0), - (vertex2_streams[1].to_string(), 1), - ], - partitions: 2, + streams: vertex2_streams.clone(), ..Default::default() }, conditions: Some(Box::new(ForwardConditions::new(TagConditions { @@ -1166,12 +1233,9 @@ mod tests { }, ToVertexConfig { name: "vertex3".to_string(), + partitions: 2, writer_config: BufferWriterConfig { - streams: vec![ - (vertex3_streams[0].to_string(), 0), - (vertex3_streams[1].to_string(), 1), - ], - partitions: 2, + streams: vertex3_streams.clone(), ..Default::default() }, conditions: Some(Box::new(ForwardConditions::new(TagConditions { @@ -1184,6 +1248,7 @@ mod tests { 100, tracker_handle.clone(), cln_token.clone(), + None, ); let (messages_tx, messages_rx) = tokio::sync::mpsc::channel(500); @@ -1193,12 +1258,13 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: Some(Arc::from(vec!["tag1".to_string(), "tag2".to_string()])), value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -1248,23 +1314,23 @@ mod tests { .chain(&vertex2_streams) .chain(&vertex3_streams) { - context.delete_stream(stream_name).await.unwrap(); + context.delete_stream(stream_name.name).await.unwrap(); } } async fn create_streams_and_consumers( context: &Context, - stream_names: &[&str], + stream_names: &[Stream], ) -> (Vec, Vec>) { let mut streams = Vec::new(); let mut consumers = Vec::new(); for stream_name in stream_names { - let _ = context.delete_stream(stream_name).await; + let _ = context.delete_stream(stream_name.name).await; let stream = context .get_or_create_stream(stream::Config { - name: stream_name.to_string(), - subjects: vec![stream_name.to_string()], + name: stream_name.name.to_string(), + subjects: vec![stream_name.name.to_string()], ..Default::default() }) .await @@ -1274,11 +1340,11 @@ mod tests { let consumer = context .create_consumer_on_stream( Config { - name: Some(stream_name.to_string()), + name: Some(stream_name.name.to_string()), ack_policy: consumer::AckPolicy::Explicit, ..Default::default() }, - stream_name, + stream_name.name, ) .await .unwrap(); diff --git a/rust/numaflow-core/src/shared/create_components.rs b/rust/numaflow-core/src/shared/create_components.rs index 593bc5ea2e..aca93d864d 100644 --- a/rust/numaflow-core/src/shared/create_components.rs +++ b/rust/numaflow-core/src/shared/create_components.rs @@ -26,6 +26,7 @@ use crate::source::user_defined::new_source; use crate::source::Source; use crate::tracker::TrackerHandle; use crate::transformer::Transformer; +use crate::watermark::source::SourceWatermarkHandle; use crate::{config, error, metrics, source}; /// Creates a sink writer based on the configuration @@ -272,6 +273,7 @@ pub async fn create_source( source_config: &SourceConfig, tracker_handle: TrackerHandle, transformer: Option, + watermark_handle: Option, cln_token: CancellationToken, ) -> error::Result<(Source, Option>)> { match &source_config.source_type { @@ -285,6 +287,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, + watermark_handle, ), None, )) @@ -324,6 +327,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, + None, ), Some(source_grpc_client), )) @@ -337,6 +341,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, + None, ), None, )) @@ -356,6 +361,7 @@ pub async fn create_source( tracker_handle, source_config.read_ahead, transformer, + None, ), None, )) diff --git a/rust/numaflow-core/src/shared/grpc.rs b/rust/numaflow-core/src/shared/grpc.rs index bedfd2e138..28c501fce5 100644 --- a/rust/numaflow-core/src/shared/grpc.rs +++ b/rust/numaflow-core/src/shared/grpc.rs @@ -102,11 +102,11 @@ pub(crate) async fn wait_until_mapper_ready( Ok(()) } -pub(crate) fn prost_timestamp_from_utc(t: DateTime) -> Option { - Some(Timestamp { +pub(crate) fn prost_timestamp_from_utc(t: DateTime) -> Timestamp { + Timestamp { seconds: t.timestamp(), nanos: t.nanosecond() as i32, - }) + } } pub(crate) async fn create_rpc_channel(socket_path: PathBuf) -> error::Result { diff --git a/rust/numaflow-core/src/sink.rs b/rust/numaflow-core/src/sink.rs index 0ae5a911dc..8d4fba9d26 100644 --- a/rust/numaflow-core/src/sink.rs +++ b/rust/numaflow-core/src/sink.rs @@ -1,3 +1,8 @@ +//! The [Sink] serves as the endpoint for processed data that has been outputted from the platform, +//! which is then sent to an external system or application. +//! +//! [Sink]: https://numaflow.numaproj.io/user-guide/sinks/overview/ + use std::collections::HashMap; use std::time::Duration; @@ -27,7 +32,7 @@ use crate::metrics::{ use crate::tracker::TrackerHandle; use crate::Result; -/// A [blackhole] sink which reads but never writes to anywhere, semantic equivalent of `/dev/null`. +/// A [Blackhole] sink which reads but never writes to anywhere, semantic equivalent of `/dev/null`. /// /// [Blackhole]: https://numaflow.numaproj.io/user-guide/sinks/blackhole/ mod blackhole; @@ -281,7 +286,7 @@ impl SinkWriter { let offsets = batch .iter() - .map(|msg| msg.id.offset.clone()) + .map(|msg| msg.offset.clone()) .collect::>(); let total_msgs = batch.len(); @@ -608,7 +613,7 @@ impl SinkWriter { // construct the error map for the failed messages messages_to_send.retain(|msg| { if let Some(result) = result_map.get(&msg.id.to_string()) { - return match result { + match result { ResponseStatusFromSink::Success => false, ResponseStatusFromSink::Failed(err_msg) => { *fallback_error_map.entry(err_msg.clone()).or_insert(0) += 1; @@ -618,7 +623,7 @@ impl SinkWriter { contains_fallback_status = true; false } - }; + } } else { false } @@ -721,7 +726,7 @@ mod tests { use tokio_util::sync::CancellationToken; use super::*; - use crate::message::{Message, MessageID, Offset, ReadAck, StringOffset}; + use crate::message::{IntOffset, Message, MessageID, Offset, ReadAck}; use crate::shared::grpc::create_rpc_channel; struct SimpleSink; @@ -752,7 +757,7 @@ mod tests { 10, Duration::from_secs(1), SinkClientType::Log, - TrackerHandle::new(None), + TrackerHandle::new(None, None), ) .build() .await @@ -763,12 +768,13 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -783,7 +789,7 @@ mod tests { #[tokio::test] async fn test_streaming_write() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let sink_writer = SinkWriterBuilder::new( 10, Duration::from_millis(100), @@ -799,12 +805,13 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -836,7 +843,7 @@ mod tests { #[tokio::test] async fn test_streaming_write_error() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); // start the server let (_shutdown_tx, shutdown_rx) = oneshot::channel(); let tmp_dir = tempfile::TempDir::new().unwrap(); @@ -875,12 +882,13 @@ mod tests { keys: Arc::from(vec!["error".to_string()]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -919,7 +927,7 @@ mod tests { #[tokio::test] async fn test_fallback_write() { - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); // start the server let (_shutdown_tx, shutdown_rx) = oneshot::channel(); @@ -960,12 +968,13 @@ mod tests { keys: Arc::from(vec!["fallback".to_string()]), tags: None, value: format!("message {}", i).as_bytes().to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(i, 0)), event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: format!("offset_{}", i).into(), - index: i, + index: i as i32, }, headers: HashMap::new(), metadata: None, @@ -1002,11 +1011,9 @@ mod tests { keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), - offset: Some(Offset::String(StringOffset { - offset: "123".to_string().into(), - partition_idx: 0, - })), + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Utc.timestamp_opt(1627846261, 0).unwrap(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "123".to_string().into(), diff --git a/rust/numaflow-core/src/sink/blackhole.rs b/rust/numaflow-core/src/sink/blackhole.rs index 328731b916..1cfb994f06 100644 --- a/rust/numaflow-core/src/sink/blackhole.rs +++ b/rust/numaflow-core/src/sink/blackhole.rs @@ -36,7 +36,7 @@ mod tests { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: Some(Offset::Int(IntOffset::new(1, 0))), + offset: Offset::Int(IntOffset::new(1, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -44,13 +44,14 @@ mod tests { offset: "1".to_string().into(), index: 0, }, + watermark: None, metadata: None, }, Message { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: Some(Offset::Int(IntOffset::new(1, 0))), + offset: Offset::Int(IntOffset::new(1, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -58,6 +59,7 @@ mod tests { offset: "2".to_string().into(), index: 1, }, + watermark: None, metadata: None, }, ]; diff --git a/rust/numaflow-core/src/sink/log.rs b/rust/numaflow-core/src/sink/log.rs index c70a1d2537..f71ca7f077 100644 --- a/rust/numaflow-core/src/sink/log.rs +++ b/rust/numaflow-core/src/sink/log.rs @@ -49,7 +49,7 @@ mod tests { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: Some(Offset::Int(IntOffset::new(1, 0))), + offset: Offset::Int(IntOffset::new(1, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -57,13 +57,14 @@ mod tests { offset: "1".to_string().into(), index: 0, }, + watermark: None, metadata: None, }, Message { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: Some(Offset::Int(IntOffset::new(1, 0))), + offset: Offset::Int(IntOffset::new(1, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -71,6 +72,7 @@ mod tests { offset: "2".to_string().into(), index: 1, }, + watermark: None, metadata: None, }, ]; diff --git a/rust/numaflow-core/src/sink/user_defined.rs b/rust/numaflow-core/src/sink/user_defined.rs index 76e0ba6ad6..b6246c2b0f 100644 --- a/rust/numaflow-core/src/sink/user_defined.rs +++ b/rust/numaflow-core/src/sink/user_defined.rs @@ -27,8 +27,8 @@ impl From for SinkRequest { request: Some(numaflow_pb::clients::sink::sink_request::Request { keys: message.keys.to_vec(), value: message.value.to_vec(), - event_time: prost_timestamp_from_utc(message.event_time), - watermark: None, + event_time: Some(prost_timestamp_from_utc(message.event_time)), + watermark: message.watermark.map(prost_timestamp_from_utc), id: message.id.to_string(), headers: message.headers, }), @@ -115,7 +115,7 @@ impl Sink for UserDefinedSink { .await? .ok_or(Error::Sink("failed to receive response".to_string()))?; - if response.status.map_or(false, |s| s.eot) { + if response.status.is_some_and(|s| s.eot) { if responses.len() != num_requests { error!("received EOT message before all responses are received, we will wait indefinitely for the remaining responses"); } else { @@ -147,7 +147,7 @@ mod tests { use super::*; use crate::error::Result; - use crate::message::{Message, MessageID}; + use crate::message::{IntOffset, Message, MessageID, Offset}; use crate::shared::grpc::create_rpc_channel; use crate::sink::user_defined::UserDefinedSink; @@ -204,7 +204,7 @@ mod tests { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -212,13 +212,14 @@ mod tests { offset: "1".to_string().into(), index: 0, }, + watermark: None, metadata: None, }, Message { keys: Arc::from(vec![]), tags: None, value: b"Hello, World!".to_vec().into(), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Utc::now(), headers: Default::default(), id: MessageID { @@ -226,6 +227,7 @@ mod tests { offset: "2".to_string().into(), index: 1, }, + watermark: None, metadata: None, }, ]; diff --git a/rust/numaflow-core/src/source.rs b/rust/numaflow-core/src/source.rs index 94235600a2..27f594f5f5 100644 --- a/rust/numaflow-core/src/source.rs +++ b/rust/numaflow-core/src/source.rs @@ -1,6 +1,23 @@ +//! [Source] vertex is responsible for reliable reading data from an unbounded source into Numaflow +//! and also assigning [Watermark]. +//! +//! [Source]: https://numaflow.numaproj.io/user-guide/sources/overview/ +//! [Watermark]: https://numaflow.numaproj.io/core-concepts/watermarks/ + use std::sync::Arc; +use numaflow_pulsar::source::PulsarSource; +use tokio::sync::OwnedSemaphorePermit; +use tokio::sync::Semaphore; +use tokio::sync::{mpsc, oneshot}; +use tokio::task::JoinHandle; +use tokio::time::Instant; +use tokio_stream::wrappers::ReceiverStream; +use tokio_util::sync::CancellationToken; +use tracing::{error, info, warn}; + use crate::config::{get_vertex_name, is_mono_vertex}; +use crate::error::{Error, Result}; use crate::message::ReadAck; use crate::metrics::{ monovertex_metrics, mvtx_forward_metric_labels, pipeline_forward_metric_labels, @@ -11,17 +28,6 @@ use crate::{ message::{Message, Offset}, reader::LagReader, }; -use crate::{Error, Result}; -use numaflow_pulsar::source::PulsarSource; -use tokio::sync::OwnedSemaphorePermit; -use tokio::sync::Semaphore; -use tokio::sync::{mpsc, oneshot}; -use tokio::task::JoinHandle; -use tokio::time; -use tokio::time::Instant; -use tokio_stream::wrappers::ReceiverStream; -use tokio_util::sync::CancellationToken; -use tracing::{error, info, warn}; /// [User-Defined Source] extends Numaflow to add custom sources supported outside the builtins. /// @@ -39,9 +45,11 @@ pub(crate) mod generator; pub(crate) mod pulsar; pub(crate) mod serving; -use crate::transformer::Transformer; use serving::ServingSource; +use crate::transformer::Transformer; +use crate::watermark::source::SourceWatermarkHandle; + /// Set of Read related items that has to be implemented to become a Source. pub(crate) trait SourceReader { #[allow(dead_code)] @@ -156,6 +164,7 @@ pub(crate) struct Source { read_ahead: bool, /// Transformer handler for transforming messages from Source. transformer: Option, + watermark_handle: Option, } impl Source { @@ -166,6 +175,7 @@ impl Source { tracker_handle: TrackerHandle, read_ahead: bool, transformer: Option, + watermark_handle: Option, ) -> Self { let (sender, receiver) = mpsc::channel(batch_size); match src_type { @@ -206,6 +216,7 @@ impl Source { tracker_handle, read_ahead, transformer, + watermark_handle, } } @@ -218,7 +229,7 @@ impl Source { let _ = source_handle.send(msg).await; receiver .await - .map_err(|e| crate::error::Error::ActorPatternRecv(e.to_string()))? + .map_err(|e| Error::ActorPatternRecv(e.to_string()))? } /// ack the offsets by communicating with the ack actor. @@ -233,7 +244,7 @@ impl Source { let _ = source_handle.send(msg).await; receiver .await - .map_err(|e| crate::error::Error::ActorPatternRecv(e.to_string()))? + .map_err(|e| Error::ActorPatternRecv(e.to_string()))? } /// get the pending messages count by communicating with the pending actor. @@ -245,7 +256,7 @@ impl Source { let _ = self.sender.send(msg).await; receiver .await - .map_err(|e| crate::error::Error::ActorPatternRecv(e.to_string()))? + .map_err(|e| Error::ActorPatternRecv(e.to_string()))? } /// Starts streaming messages from the source. It returns a stream of messages and @@ -260,6 +271,7 @@ impl Source { let tracker_handle = self.tracker_handle.clone(); let read_ahead_enabled = self.read_ahead; let mut transformer = self.transformer.clone(); + let mut watermark_handle = self.watermark_handle.clone(); let pipeline_labels = pipeline_forward_metric_labels("Source", Some(get_vertex_name())); let mvtx_labels = mvtx_forward_metric_labels(); @@ -285,7 +297,7 @@ impl Source { .expect("acquiring permit should not fail"); } - let read_start_time = time::Instant::now(); + let read_start_time = Instant::now(); let messages = match Self::read(source_handle.clone()).await { Ok(messages) => messages, Err(e) => { @@ -305,7 +317,7 @@ impl Source { let mut ack_batch = Vec::with_capacity(msgs_len); for message in messages.iter() { let (resp_ack_tx, resp_ack_rx) = oneshot::channel(); - let offset = message.offset.clone().expect("offset can never be none"); + let offset = message.offset.clone(); // insert the offset and the ack one shot in the tracker. tracker_handle.insert(message, resp_ack_tx).await?; @@ -335,6 +347,12 @@ impl Source { Some(transformer) => transformer.transform_batch(messages).await?, }; + if let Some(watermark_handle) = watermark_handle.as_mut() { + watermark_handle + .generate_and_publish_source_watermark(&messages) + .await?; + } + // write the messages to downstream. for message in messages { messages_tx.send(message).await.map_err(|e| { @@ -567,9 +585,10 @@ mod tests { let source = Source::new( 5, SourceType::UserDefinedSource(src_read, src_ack, lag_reader), - TrackerHandle::new(None), + TrackerHandle::new(None, None), true, None, + None, ); let cln_token = CancellationToken::new(); @@ -580,7 +599,7 @@ mod tests { for _ in 0..100 { let message = stream.next().await.unwrap(); assert_eq!(message.value, "hello".as_bytes()); - offsets.push(message.offset.clone().unwrap()); + offsets.push(message.offset.clone()); } // ack all the messages diff --git a/rust/numaflow-core/src/source/generator.rs b/rust/numaflow-core/src/source/generator.rs index b1ab1694d3..97cdc8d20b 100644 --- a/rust/numaflow-core/src/source/generator.rs +++ b/rust/numaflow-core/src/source/generator.rs @@ -35,7 +35,7 @@ mod stream_generator { use crate::config::components::source::GeneratorConfig; use crate::config::{get_vertex_name, get_vertex_replica}; - use crate::message::{Message, MessageID, Offset, StringOffset}; + use crate::message::{IntOffset, Message, MessageID, Offset}; #[pin_project] pub(super) struct StreamGenerator { /// the content generated by Generator. @@ -145,12 +145,9 @@ mod stream_generator { /// creates a single message that can be returned by the generator. fn create_message(&mut self) -> Message { - let id = chrono::Utc::now() - .timestamp_nanos_opt() - .unwrap_or_default() - .to_string(); + let id = chrono::Utc::now().timestamp_nanos_opt().unwrap_or_default(); - let offset = Offset::String(StringOffset::new(id.clone(), *get_vertex_replica())); + let offset = Offset::Int(IntOffset::new(id, *get_vertex_replica())); // rng.gen_range(0..0) panics with "cannot sample empty range" // rng.gen_range(0..1) will always produce 0 @@ -170,8 +167,9 @@ mod stream_generator { keys: Arc::from(self.next_key_to_be_fetched()), tags: None, value: data.into(), - offset: Some(offset.clone()), + offset: offset.clone(), event_time, + watermark: None, id: MessageID { vertex_name: get_vertex_name().to_string().into(), offset: offset.to_string().into(), @@ -393,7 +391,7 @@ mod tests { use tokio::time::Duration; use super::*; - use crate::message::StringOffset; + use crate::message::IntOffset; use crate::reader::LagReader; use crate::source::{SourceAcker, SourceReader}; @@ -488,8 +486,8 @@ mod tests { // Create a vector of offsets to acknowledge let offsets = vec![ - Offset::String(StringOffset::new("offset1".to_string(), 0)), - Offset::String(StringOffset::new("offset2".to_string(), 0)), + Offset::Int(IntOffset::new(1, 0)), + Offset::Int(IntOffset::new(2, 0)), ]; // Call the ack method and check the result diff --git a/rust/numaflow-core/src/source/pulsar.rs b/rust/numaflow-core/src/source/pulsar.rs index 5e02db3da6..0c32598a17 100644 --- a/rust/numaflow-core/src/source/pulsar.rs +++ b/rust/numaflow-core/src/source/pulsar.rs @@ -3,7 +3,7 @@ use std::time::Duration; use numaflow_pulsar::source::{PulsarMessage, PulsarSource, PulsarSourceConfig}; -use crate::config::get_vertex_name; +use crate::config::{get_vertex_name, get_vertex_replica}; use crate::error::Error; use crate::message::{IntOffset, Message, MessageID, Offset}; use crate::source; @@ -12,14 +12,15 @@ impl TryFrom for Message { type Error = Error; fn try_from(message: PulsarMessage) -> crate::Result { - let offset = Offset::Int(IntOffset::new(message.offset, 1)); // FIXME: partition id + let offset = Offset::Int(IntOffset::new(message.offset as i64, *get_vertex_replica())); Ok(Message { keys: Arc::from(vec![message.key]), tags: None, value: message.payload, - offset: Some(offset.clone()), + offset: offset.clone(), event_time: message.event_time, + watermark: None, id: MessageID { vertex_name: get_vertex_name().to_string().into(), offset: offset.to_string().into(), @@ -82,7 +83,7 @@ impl source::SourceAcker for PulsarSource { "Expected Offset::Int type for Pulsar. offset={offset:?}" ))); }; - pulsar_offsets.push(int_offset.offset); + pulsar_offsets.push(int_offset.offset as u64); } self.ack_offsets(pulsar_offsets).await.map_err(Into::into) } @@ -154,7 +155,7 @@ mod tests { let messages = pulsar.read().await?; assert_eq!(messages.len(), 10); - let offsets: Vec = messages.into_iter().map(|m| m.offset.unwrap()).collect(); + let offsets: Vec = messages.into_iter().map(|m| m.offset).collect(); pulsar.ack(offsets).await?; diff --git a/rust/numaflow-core/src/source/serving.rs b/rust/numaflow-core/src/source/serving.rs index 65f3a3e153..3c1e8bc77a 100644 --- a/rust/numaflow-core/src/source/serving.rs +++ b/rust/numaflow-core/src/source/serving.rs @@ -19,8 +19,9 @@ impl TryFrom for Message { keys: Arc::from(vec![]), tags: None, value: message.value, - offset: Some(offset.clone()), + offset: offset.clone(), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: get_vertex_name().to_string().into(), offset: offset.to_string().into(), @@ -91,10 +92,8 @@ mod tests { use serving::{ServingSource, Settings}; use super::get_vertex_replica; - use crate::{ - message::{Message, MessageID, Offset, StringOffset}, - source::{SourceAcker, SourceReader}, - }; + use crate::message::{Message, MessageID, Offset, StringOffset}; + use crate::source::{SourceAcker, SourceReader}; type Result = std::result::Result>; @@ -114,7 +113,7 @@ mod tests { assert_eq!(message.value, Bytes::from_static(b"test")); assert_eq!( message.offset, - Some(Offset::String(StringOffset::new(MSG_ID.into(), 0))) + Offset::String(StringOffset::new(MSG_ID.into(), 0)) ); assert_eq!( message.id, @@ -192,7 +191,7 @@ mod tests { } assert_eq!(messages.len(), 1); let msg = messages.remove(0); - serving_source.ack(vec![msg.offset.unwrap()]).await.unwrap(); + serving_source.ack(vec![msg.offset]).await.unwrap(); break; } }); diff --git a/rust/numaflow-core/src/source/user_defined.rs b/rust/numaflow-core/src/source/user_defined.rs index d089e95acb..374a9ead66 100644 --- a/rust/numaflow-core/src/source/user_defined.rs +++ b/rust/numaflow-core/src/source/user_defined.rs @@ -121,7 +121,7 @@ impl TryFrom for Message { keys: Arc::from(result.keys), tags: None, value: result.payload.into(), - offset: Some(source_offset.clone()), + offset: source_offset.clone(), event_time: utc_from_timestamp(result.event_time), id: MessageID { vertex_name: config::get_vertex_name().to_string().into(), @@ -129,6 +129,7 @@ impl TryFrom for Message { index: 0, }, headers: result.headers, + watermark: None, metadata: None, }) } @@ -139,13 +140,16 @@ impl TryFrom for source::Offset { fn try_from(offset: Offset) -> std::result::Result { match offset { - Offset::Int(_) => Err(Error::Source("IntOffset not supported".to_string())), - Offset::String(o) => Ok(numaflow_pb::clients::source::Offset { + Offset::String(StringOffset { + offset, + partition_idx, + }) => Ok(source::Offset { offset: BASE64_STANDARD - .decode(o.offset) + .decode(offset) .expect("we control the encoding, so this should never fail"), - partition_id: o.partition_idx as i32, + partition_id: partition_idx as i32, }), + Offset::Int(_) => Err(Error::Source("IntOffset not supported".to_string())), } } } @@ -172,7 +176,7 @@ impl SourceReader for UserDefinedSourceRead { let mut messages = Vec::with_capacity(self.num_records); while let Some(response) = self.resp_stream.message().await? { - if response.status.map_or(false, |status| status.eot) { + if response.status.is_some_and(|status| status.eot) { break; } @@ -387,7 +391,7 @@ mod tests { assert_eq!(messages.len(), 5); let response = src_ack - .ack(messages.iter().map(|m| m.offset.clone().unwrap()).collect()) + .ack(messages.iter().map(|m| m.offset.clone()).collect()) .await; assert!(response.is_ok()); @@ -413,9 +417,9 @@ mod tests { offset: BASE64_STANDARD.encode("123").into_bytes(), partition_id: 0, }), - event_time: Some( - prost_timestamp_from_utc(Utc.timestamp_opt(1627846261, 0).unwrap()).unwrap(), - ), + event_time: Some(prost_timestamp_from_utc( + Utc.timestamp_opt(1627846261, 0).unwrap(), + )), keys: vec!["key1".to_string()], headers: HashMap::new(), }; diff --git a/rust/numaflow-core/src/tracker.rs b/rust/numaflow-core/src/tracker.rs index acdfe7d2c2..67672123df 100644 --- a/rust/numaflow-core/src/tracker.rs +++ b/rust/numaflow-core/src/tracker.rs @@ -6,18 +6,22 @@ //! NAck otherwise if ISB is failing to accept, and we are in shutdown path. //! There will be a tracker per input stream reader. //! -//! In the future Watermark will also be propagated based on this. +//! Items tracked by the tracker and uses [Offset] as the key. +//! - Ack or NAck after processing of a message +//! - The oldest Watermark is tracked +//! - Callbacks for Serving is triggered in the tracker. use std::collections::HashMap; use std::sync::Arc; -use bytes::Bytes; +use chrono::{DateTime, Utc}; use serving::callback::CallbackHandler; use serving::{DEFAULT_CALLBACK_URL_HEADER_KEY, DEFAULT_ID_HEADER}; use tokio::sync::{mpsc, oneshot}; use crate::error::Error; -use crate::message::{Message, ReadAck}; +use crate::message::{Message, Offset, ReadAck}; +use crate::watermark::isb::ISBWatermarkHandle; use crate::Result; /// TrackerEntry represents the state of a tracked message. @@ -36,22 +40,23 @@ struct TrackerEntry { /// ActorMessage represents the messages that can be sent to the Tracker actor. enum ActorMessage { Insert { - offset: Bytes, + offset: Offset, ack_send: oneshot::Sender, serving_callback_info: Option, + watermark: Option>, }, Update { - offset: Bytes, + offset: Offset, responses: Option>, }, - UpdateEOF { - offset: Bytes, - }, Delete { - offset: Bytes, + offset: Offset, }, Discard { - offset: Bytes, + offset: Offset, + }, + UpdateEOF { + offset: Offset, }, DiscardAll, // New variant for discarding all messages #[cfg(test)] @@ -63,9 +68,10 @@ enum ActorMessage { /// Tracker is responsible for managing the state of messages being processed. /// It keeps track of message offsets and their completeness, and sends acknowledgments. struct Tracker { - /// number of entri - entries: HashMap, + /// number of entries in the tracker + entries: HashMap, receiver: mpsc::Receiver, + watermark_handle: Option, serving_callback_handler: Option, } @@ -139,11 +145,13 @@ impl Tracker { /// Creates a new Tracker instance with the given receiver for actor messages. fn new( receiver: mpsc::Receiver, + watermark_handle: Option, serving_callback_handler: Option, ) -> Self { Self { entries: HashMap::new(), receiver, + watermark_handle, serving_callback_handler, } } @@ -162,8 +170,10 @@ impl Tracker { offset, ack_send: respond_to, serving_callback_info: callback_info, + watermark, } => { - self.handle_insert(offset, callback_info, respond_to); + self.handle_insert(offset, callback_info, watermark, respond_to) + .await; } ActorMessage::Update { offset, responses } => { self.handle_update(offset, responses); @@ -175,7 +185,7 @@ impl Tracker { self.handle_delete(offset).await; } ActorMessage::Discard { offset } => { - self.handle_discard(offset); + self.handle_discard(offset).await; } ActorMessage::DiscardAll => { self.handle_discard_all().await; @@ -189,14 +199,15 @@ impl Tracker { } /// Inserts a new entry into the tracker with the given offset and ack sender. - fn handle_insert( + async fn handle_insert( &mut self, - offset: Bytes, + offset: Offset, callback_info: Option, + watermark: Option>, respond_to: oneshot::Sender, ) { self.entries.insert( - offset, + offset.clone(), TrackerEntry { ack_send: respond_to, count: 0, @@ -204,10 +215,17 @@ impl Tracker { serving_callback_info: callback_info, }, ); + + if let Some(watermark_handle) = &self.watermark_handle { + watermark_handle + .insert_offset(offset, watermark) + .await + .expect("Failed to insert offset"); + } } /// Updates an existing entry in the tracker with the number of expected messages for this offset. - fn handle_update(&mut self, offset: Bytes, responses: Option>) { + fn handle_update(&mut self, offset: Offset, responses: Option>) { let Some(entry) = self.entries.get_mut(&offset) else { return; }; @@ -219,7 +237,7 @@ impl Tracker { } /// Update whether we have seen the eof (end of stream) for this offset. - async fn handle_update_eof(&mut self, offset: Bytes) { + async fn handle_update_eof(&mut self, offset: Offset) { let Some(entry) = self.entries.get_mut(&offset) else { return; }; @@ -229,13 +247,13 @@ impl Tracker { // receiving all the messages. if entry.count == 0 { let entry = self.entries.remove(&offset).unwrap(); - self.ack_message(entry).await; + self.completed_successfully(offset, entry).await; } } /// Removes an entry from the tracker and sends an acknowledgment if the count is zero /// and the entry is marked as EOF. - async fn handle_delete(&mut self, offset: Bytes) { + async fn handle_delete(&mut self, offset: Offset) { let Some(mut entry) = self.entries.remove(&offset) else { return; }; @@ -247,7 +265,7 @@ impl Tracker { // In map-streaming this won't happen because eof is not tied to the message, rather it is // tied to channel-close. if entry.count == 0 && entry.eof { - self.ack_message(entry).await; + self.completed_successfully(offset, entry).await; } else { // add it back because we removed it self.entries.insert(offset, entry); @@ -255,7 +273,7 @@ impl Tracker { } /// Discards an entry from the tracker and sends a nak. - fn handle_discard(&mut self, offset: Bytes) { + async fn handle_discard(&mut self, offset: Offset) { let Some(entry) = self.entries.remove(&offset) else { return; }; @@ -263,6 +281,12 @@ impl Tracker { .ack_send .send(ReadAck::Nak) .expect("Failed to send nak"); + if let Some(watermark_handle) = &self.watermark_handle { + watermark_handle + .remove_offset(offset) + .await + .expect("Failed to remove offset"); + } } /// Discards all entries from the tracker and sends a nak for each. @@ -280,7 +304,7 @@ impl Tracker { /// - ack back /// - call serving callbacks /// - watermark progression - async fn ack_message(&self, entry: TrackerEntry) { + async fn completed_successfully(&self, offset: Offset, entry: TrackerEntry) { let TrackerEntry { ack_send, serving_callback_info: callback_info, @@ -289,6 +313,13 @@ impl Tracker { ack_send.send(ReadAck::Ack).expect("Failed to send ack"); + if let Some(watermark_handle) = &self.watermark_handle { + watermark_handle + .remove_offset(offset) + .await + .expect("Failed to remove offset"); + } + let Some(ref callback_handler) = self.serving_callback_handler else { return; }; @@ -322,10 +353,13 @@ pub(crate) struct TrackerHandle { impl TrackerHandle { /// Creates a new TrackerHandle instance and spawns the Tracker. - pub(crate) fn new(callback_handler: Option) -> Self { + pub(crate) fn new( + watermark_handle: Option, + callback_handler: Option, + ) -> Self { let enable_callbacks = callback_handler.is_some(); let (sender, receiver) = mpsc::channel(100); - let tracker = Tracker::new(receiver, callback_handler); + let tracker = Tracker::new(receiver, watermark_handle, callback_handler); tokio::spawn(tracker.run()); Self { sender, @@ -339,7 +373,7 @@ impl TrackerHandle { message: &Message, ack_send: oneshot::Sender, ) -> Result<()> { - let offset = message.id.offset.clone(); + let offset = message.offset.clone(); let mut callback_info = None; if self.enable_callbacks { callback_info = Some(message.try_into()?); @@ -348,6 +382,7 @@ impl TrackerHandle { offset, ack_send, serving_callback_info: callback_info, + watermark: message.watermark, }; self.sender .send(message) @@ -360,7 +395,7 @@ impl TrackerHandle { /// and entry for this message's offset. pub(crate) async fn update( &self, - offset: Bytes, + offset: Offset, message_tags: Option>, ) -> Result<()> { let responses: Option> = match (self.enable_callbacks, message_tags) { @@ -382,7 +417,7 @@ impl TrackerHandle { } /// Updates the EOF status for an offset in the Tracker - pub(crate) async fn update_eof(&self, offset: Bytes) -> Result<()> { + pub(crate) async fn update_eof(&self, offset: Offset) -> Result<()> { let message = ActorMessage::UpdateEOF { offset }; self.sender .send(message) @@ -392,7 +427,7 @@ impl TrackerHandle { } /// Deletes a message from the Tracker with the given offset. - pub(crate) async fn delete(&self, offset: Bytes) -> Result<()> { + pub(crate) async fn delete(&self, offset: Offset) -> Result<()> { let message = ActorMessage::Delete { offset }; self.sender .send(message) @@ -402,7 +437,7 @@ impl TrackerHandle { } /// Discards a message from the Tracker with the given offset. - pub(crate) async fn discard(&self, offset: Bytes) -> Result<()> { + pub(crate) async fn discard(&self, offset: Offset) -> Result<()> { let message = ActorMessage::Discard { offset }; self.sender .send(message) @@ -443,27 +478,28 @@ mod tests { use axum::routing::{get, post}; use axum::{http::StatusCode, Router}; + use bytes::Bytes; use tokio::sync::oneshot; use tokio::time::{timeout, Duration}; - use crate::message::{MessageID, Metadata}; - use super::*; + use crate::message::StringOffset; + use crate::message::{IntOffset, MessageID, Metadata, Offset}; type Result = std::result::Result>; #[test] fn test_message_to_callback_info_conversion() { - let offset = Bytes::from_static(b"offset1"); let mut message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"0"), index: 1, }, headers: HashMap::new(), @@ -483,7 +519,7 @@ mod tests { .collect(); message.headers = headers; - const FROM_VERTEX_NAME: &str = "source-vetext"; + const FROM_VERTEX_NAME: &str = "source-vertex"; message.metadata = Some(Metadata { previous_vertex: FROM_VERTEX_NAME.into(), }); @@ -497,19 +533,19 @@ mod tests { #[tokio::test] async fn test_insert_update_delete() { - let handle = TrackerHandle::new(None); + let handle = TrackerHandle::new(None, None); let (ack_send, ack_recv) = oneshot::channel(); - let offset = Bytes::from_static(b"offset1"); let message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: Offset::String(StringOffset::new("offset1".to_string(), 0)), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"offset1"), index: 1, }, headers: HashMap::new(), @@ -521,13 +557,13 @@ mod tests { // Update the message handle - .update(offset.clone(), message.tags.clone()) + .update(message.offset.clone(), message.tags.clone()) .await .unwrap(); - handle.update_eof(offset).await.unwrap(); + handle.update_eof(message.offset.clone()).await.unwrap(); // Delete the message - handle.delete("offset1".to_string().into()).await.unwrap(); + handle.delete(message.offset).await.unwrap(); // Verify that the message was deleted and ack was received let result = timeout(Duration::from_secs(1), ack_recv).await.unwrap(); @@ -538,19 +574,18 @@ mod tests { #[tokio::test] async fn test_update_with_multiple_deletes() { - let handle = TrackerHandle::new(None); + let handle = TrackerHandle::new(None, None); let (ack_send, ack_recv) = oneshot::channel(); - - let offset = Bytes::from_static(b"offset1"); let message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: Offset::String(StringOffset::new("offset1".to_string(), 0)), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"offset1"), index: 1, }, headers: HashMap::new(), @@ -560,19 +595,19 @@ mod tests { // Insert a new message handle.insert(&message, ack_send).await.unwrap(); - let messages: Vec = std::iter::repeat(message).take(3).collect(); + let messages: Vec = std::iter::repeat(message.clone()).take(3).collect(); // Update the message with a count of 3 for message in messages { handle - .update(offset.clone(), message.tags.clone()) + .update(message.offset.clone(), message.tags.clone()) .await .unwrap(); } // Delete the message three times - handle.delete(offset.clone()).await.unwrap(); - handle.delete(offset.clone()).await.unwrap(); - handle.delete(offset).await.unwrap(); + handle.delete(message.offset.clone()).await.unwrap(); + handle.delete(message.offset.clone()).await.unwrap(); + handle.delete(message.offset.clone()).await.unwrap(); // Verify that the message was deleted and ack was received after the third delete let result = timeout(Duration::from_secs(1), ack_recv).await.unwrap(); @@ -583,19 +618,19 @@ mod tests { #[tokio::test] async fn test_discard() { - let handle = TrackerHandle::new(None); + let handle = TrackerHandle::new(None, None); let (ack_send, ack_recv) = oneshot::channel(); - let offset = Bytes::from_static(b"offset1"); let message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"0"), index: 1, }, headers: HashMap::new(), @@ -606,7 +641,7 @@ mod tests { handle.insert(&message, ack_send).await.unwrap(); // Discard the message - handle.discard(offset).await.unwrap(); + handle.discard(message.offset.clone()).await.unwrap(); // Verify that the message was discarded and nak was received let result = timeout(Duration::from_secs(1), ack_recv).await.unwrap(); @@ -617,19 +652,19 @@ mod tests { #[tokio::test] async fn test_discard_after_update_with_higher_count() { - let handle = TrackerHandle::new(None); + let handle = TrackerHandle::new(None, None); let (ack_send, ack_recv) = oneshot::channel(); - let offset = Bytes::from_static(b"offset1"); let message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: Offset::Int(IntOffset::new(0, 0)), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"0"), index: 1, }, headers: HashMap::new(), @@ -639,16 +674,16 @@ mod tests { // Insert a new message handle.insert(&message, ack_send).await.unwrap(); - let messages: Vec = std::iter::repeat(message).take(3).collect(); + let messages: Vec = std::iter::repeat(message.clone()).take(3).collect(); for message in messages { handle - .update(offset.clone(), message.tags.clone()) + .update(message.offset.clone(), message.tags.clone()) .await .unwrap(); } // Discard the message - handle.discard(offset).await.unwrap(); + handle.discard(message.offset).await.unwrap(); // Verify that the message was discarded and nak was received let result = timeout(Duration::from_secs(1), ack_recv).await.unwrap(); @@ -707,7 +742,7 @@ mod tests { assert!(server_ready, "Server is not ready"); let callback_handler = CallbackHandler::new("test".into(), 10); - let handle = TrackerHandle::new(Some(callback_handler)); + let handle = TrackerHandle::new(None, Some(callback_handler)); let (ack_send, ack_recv) = oneshot::channel(); let headers = [ @@ -718,16 +753,17 @@ mod tests { .map(|(k, v)| (k.to_string(), v.to_string())) .collect(); - let offset = Bytes::from_static(b"offset1"); + let offset = Offset::String(StringOffset::new("offset1".to_string(), 0)); let message = Message { keys: Arc::from([]), tags: None, value: Bytes::from_static(b"test"), - offset: None, + offset: offset.clone(), event_time: Default::default(), + watermark: None, id: MessageID { vertex_name: "in".into(), - offset: offset.clone(), + offset: Bytes::from_static(b"offset1"), index: 1, }, headers, diff --git a/rust/numaflow-core/src/transformer.rs b/rust/numaflow-core/src/transformer.rs index 05c567c64b..f49a473819 100644 --- a/rust/numaflow-core/src/transformer.rs +++ b/rust/numaflow-core/src/transformer.rs @@ -1,15 +1,16 @@ use std::sync::Arc; +use numaflow_pb::clients::sourcetransformer::source_transform_client::SourceTransformClient; +use tokio::sync::{mpsc, oneshot, Semaphore}; +use tonic::transport::Channel; +use tracing::info; + use crate::error::Error; use crate::message::Message; use crate::metrics::{monovertex_metrics, mvtx_forward_metric_labels}; use crate::tracker::TrackerHandle; use crate::transformer::user_defined::UserDefinedTransformer; use crate::Result; -use numaflow_pb::clients::sourcetransformer::source_transform_client::SourceTransformClient; -use tokio::sync::{mpsc, oneshot, Semaphore}; -use tonic::transport::Channel; -use tracing::info; /// User-Defined Transformer is a custom transformer that can be built by the user. /// @@ -148,12 +149,10 @@ impl Transformer { // update the tracker with the number of responses for each message for message in transformed_messages.iter() { tracker_handle - .update(read_msg.id.offset.clone(), message.tags.clone()) + .update(read_msg.offset.clone(), message.tags.clone()) .await?; } - tracker_handle - .update_eof(read_msg.id.offset.clone()) - .await?; + tracker_handle.update_eof(read_msg.offset.clone()).await?; Ok::, Error>(transformed_messages) }) @@ -221,7 +220,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = SourceTransformClient::new(create_rpc_channel(sock_file).await?); let transformer = Transformer::new(500, 10, client, tracker_handle.clone()).await?; @@ -230,8 +229,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -245,7 +245,7 @@ mod tests { Transformer::transform(transformer.sender.clone(), message).await; assert!(transformed_messages.is_ok()); - let transformed_messages = transformed_messages.unwrap(); + let transformed_messages = transformed_messages?; assert_eq!(transformed_messages.len(), 1); assert_eq!(transformed_messages[0].value, "hello"); @@ -285,7 +285,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = SourceTransformClient::new(create_rpc_channel(sock_file).await?); let transformer = Transformer::new(500, 10, client, tracker_handle.clone()).await?; @@ -295,8 +295,9 @@ mod tests { keys: Arc::from(vec![format!("key_{}", i)]), tags: None, value: format!("value_{}", i).into(), - offset: Some(Offset::String(StringOffset::new(i.to_string(), 0))), + offset: Offset::String(StringOffset::new(i.to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: i.to_string().into(), @@ -361,7 +362,7 @@ mod tests { // wait for the server to start tokio::time::sleep(Duration::from_millis(100)).await; - let tracker_handle = TrackerHandle::new(None); + let tracker_handle = TrackerHandle::new(None, None); let client = SourceTransformClient::new(create_rpc_channel(sock_file).await?); let transformer = Transformer::new(500, 10, client, tracker_handle.clone()).await?; @@ -369,8 +370,9 @@ mod tests { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(Offset::String(StringOffset::new("0".to_string(), 0))), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), event_time: chrono::Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), diff --git a/rust/numaflow-core/src/transformer/user_defined.rs b/rust/numaflow-core/src/transformer/user_defined.rs index c56afecbd5..e2bdb558b8 100644 --- a/rust/numaflow-core/src/transformer/user_defined.rs +++ b/rust/numaflow-core/src/transformer/user_defined.rs @@ -43,14 +43,11 @@ impl From for SourceTransformRequest { fn from(message: Message) -> Self { Self { request: Some(sourcetransformer::source_transform_request::Request { - id: message - .offset - .expect("offset should be present") - .to_string(), + id: message.offset.to_string(), keys: message.keys.to_vec(), value: message.value.to_vec(), - event_time: prost_timestamp_from_utc(message.event_time), - watermark: None, + event_time: Some(prost_timestamp_from_utc(message.event_time)), + watermark: message.watermark.map(prost_timestamp_from_utc), headers: message.headers, }), handshake: None, @@ -140,9 +137,10 @@ impl UserDefinedTransformer { keys: Arc::from(result.keys), tags: Some(Arc::from(result.tags)), value: result.value.into(), - offset: Some(msg_info.offset.clone()), + offset: msg_info.offset.clone(), event_time: utc_from_timestamp(result.event_time), headers: msg_info.headers.clone(), + watermark: None, metadata: None, }; response_messages.push(message); @@ -160,14 +158,10 @@ impl UserDefinedTransformer { message: Message, respond_to: oneshot::Sender>>, ) { - let key = message - .offset - .clone() - .expect("offset should be present") - .to_string(); + let key = message.offset.clone().to_string(); let msg_info = ParentMessageInfo { - offset: message.offset.clone().expect("offset can never be none"), + offset: message.offset.clone(), headers: message.headers.clone(), }; @@ -205,7 +199,7 @@ mod tests { &self, input: sourcetransform::SourceTransformRequest, ) -> Vec { - let message = sourcetransform::Message::new(input.value, chrono::offset::Utc::now()) + let message = sourcetransform::Message::new(input.value, Utc::now()) .keys(input.keys) .tags(vec![]); vec![message] @@ -214,7 +208,7 @@ mod tests { #[tokio::test] async fn transformer_operations() -> Result<(), Box> { - let (shutdown_tx, shutdown_rx) = tokio::sync::oneshot::channel(); + let (shutdown_tx, shutdown_rx) = oneshot::channel(); let tmp_dir = TempDir::new()?; let sock_file = tmp_dir.path().join("sourcetransform.sock"); let server_info_file = tmp_dir.path().join("sourcetransformer-server-info"); @@ -239,15 +233,13 @@ mod tests { ) .await?; - let message = crate::message::Message { + let message = Message { keys: Arc::from(vec!["first".into()]), tags: None, value: "hello".into(), - offset: Some(crate::message::Offset::String(StringOffset::new( - "0".to_string(), - 0, - ))), - event_time: chrono::Utc::now(), + offset: Offset::String(StringOffset::new("0".to_string(), 0)), + event_time: Utc::now(), + watermark: None, id: MessageID { vertex_name: "vertex_name".to_string().into(), offset: "0".to_string().into(), @@ -257,7 +249,7 @@ mod tests { metadata: None, }; - let (tx, rx) = tokio::sync::oneshot::channel(); + let (tx, rx) = oneshot::channel(); tokio::time::timeout(Duration::from_secs(2), client.transform(message, tx)) .await @@ -288,11 +280,12 @@ mod tests { keys: Arc::from(vec!["key1".to_string()]), tags: None, value: vec![1, 2, 3].into(), - offset: Some(Offset::String(StringOffset { + offset: Offset::String(StringOffset { offset: "123".to_string().into(), partition_idx: 0, - })), + }), event_time: Utc.timestamp_opt(1627846261, 0).unwrap(), + watermark: None, id: MessageID { vertex_name: "vertex".to_string().into(), offset: "123".to_string().into(), diff --git a/rust/numaflow-core/src/watermark.rs b/rust/numaflow-core/src/watermark.rs new file mode 100644 index 0000000000..a469428436 --- /dev/null +++ b/rust/numaflow-core/src/watermark.rs @@ -0,0 +1,50 @@ +//! The [Watermark] is propagated across all Vertices from the [crate::source] all the way to +//! the [crate::sink]. The watermark is generated at the [crate::source] based +//! on the event-time, and if it is missing, it will be the ingestion time (the time the data was +//! inserted into the pipeline). Once the watermark is created, it just flows through the subsequent +//! vertices of the pipeline. +//! Every vertex (Source, UDFs, Sink) should implement two interfaces +//! - Publish Watermark - this will be per unit per-vertex information +//! - Fetch Watermark - this will be the merged WM of the vertex across all units +//! +//! ## Publish Watermark +//! Publish publishes the watermark of the [processor] of the vertex, always the smallest watermark +//! is published (WM < the oldest message still being processed). We cannot measure the WM of the vertex +//! by looking at the WM of a [processor] of the vertex but by looking into all the WMs published +//! by all the [processor] in that vertex. +//! +//! ## Fetch Watermark +//! Fetch at vertex Vn will fetch the watermarks of all the [processor] of the vertex Vn-1. Fetch +//! happens for every message (looked up using offset) read from the stream (source or isb). We cannot +//! measure the WM of the vertex by looking at the WM of a single [processor] of the vertex but by looking +//! into all the WMs published by all the [processor] in that vertex. The vertex Vn to determine the +//! watermark of an offset will have to query the watermarks of each [processor] of the vertex Vn-1. The +//! lowest watermark among all the [processor] will be set as the watermark of that offset. +//! +//! [Watermark]: https://numaflow.numaproj.io/core-concepts/watermarks/ + +use crate::watermark::isb::ISBWatermarkHandle; +use crate::watermark::source::SourceWatermarkHandle; + +/// Responsible for fetch/publish cycle of watermark per offset for each stream in the ISB. +pub(crate) mod isb; + +/// Manages the processors for watermark. Processor is specific to watermark. +mod processor; + +/// Responsible for fetching and publishing watermarks for the Source. A Source could have multiple +/// partitions, similar partitions in the ISB. The main difference between Source and [isb] is that +/// the watermark starts at source, so we will have to do a publishing followed by a fetch and publish. +pub(crate) mod source; + +/// Stores WMB related data. +mod wmb; + +/// Watermark handle, enum to hold both edge and source watermark handles +/// This is used to fetch and publish watermarks +#[derive(Clone)] +pub(crate) enum WatermarkHandle { + #[allow(clippy::upper_case_acronyms)] + ISB(ISBWatermarkHandle), + Source(SourceWatermarkHandle), +} diff --git a/rust/numaflow-core/src/watermark/isb.rs b/rust/numaflow-core/src/watermark/isb.rs new file mode 100644 index 0000000000..6521a42129 --- /dev/null +++ b/rust/numaflow-core/src/watermark/isb.rs @@ -0,0 +1,600 @@ +//! Exposes methods to fetch the watermark for the messages read from [crate::pipeline::isb], and +//! publish the watermark for the messages written to [crate::pipeline::isb]. Manages the timelines +//! of the watermark published by the previous vertices for each partition and fetches the lowest +//! watermark among them. It tracks the watermarks of all the inflight messages for each partition, +//! and publishes the lowest watermark. The watermark published to the ISB will always be monotonically +//! increasing. Fetch and publish will be two different flows, but we will have natural ordering +//! because we use actor model. Since we do streaming within the vertex we have to track the +//! messages so that even if any messages get stuck we consider them while publishing watermarks. +//! +//! +//! **Fetch Flow** +//! ```text +//! (Read from ISB) -------> (Fetch Watermark) -------> (Track Offset and WM) +//! ``` +//! +//! **Publish Flow** +//! ```text +//! (Write to ISB) -------> (Publish Watermark) ------> (Remove tracked Offset) +//! ``` + +use std::cmp::Ordering; +use std::collections::{BTreeSet, HashMap}; + +use tokio::sync::mpsc::Receiver; +use tracing::error; + +use crate::config::pipeline::isb::Stream; +use crate::config::pipeline::watermark::EdgeWatermarkConfig; +use crate::error::{Error, Result}; +use crate::message::{IntOffset, Offset}; +use crate::watermark::isb::wm_fetcher::ISBWatermarkFetcher; +use crate::watermark::isb::wm_publisher::ISBWatermarkPublisher; +use crate::watermark::processor::manager::ProcessorManager; +use crate::watermark::wmb::Watermark; + +pub(crate) mod wm_fetcher; +pub(crate) mod wm_publisher; + +/// Messages that can be sent to the [ISBWatermarkActor]. +enum ISBWaterMarkActorMessage { + FetchWatermark { + offset: IntOffset, + oneshot_tx: tokio::sync::oneshot::Sender>, + }, + PublishWatermark { + offset: IntOffset, + stream: Stream, + }, + RemoveOffset(IntOffset), + InsertOffset { + offset: IntOffset, + watermark: Watermark, + }, +} + +/// Tuple of offset and watermark. We will use this to track the inflight messages. +#[derive(Eq, PartialEq)] +struct OffsetWatermark { + /// offset can be -1 if watermark cannot be derived. + offset: i64, + watermark: Watermark, +} + +/// Ordering will be based on the offset in OffsetWatermark +impl Ord for OffsetWatermark { + fn cmp(&self, other: &Self) -> Ordering { + self.offset.cmp(&other.offset) + } +} + +impl PartialOrd for OffsetWatermark { + fn partial_cmp(&self, other: &Self) -> Option { + Some(self.cmp(other)) + } +} + +/// EdgeWatermarkActor comprises EdgeFetcher and EdgePublisher. +/// Tracks the watermarks of all the inflight messages for each partition, and publishes +/// the lowest watermark. +struct ISBWatermarkActor { + fetcher: ISBWatermarkFetcher, + publisher: ISBWatermarkPublisher, + /// BTreeSet is used to track the watermarks of the inflight messages because we frequently + /// need to get the lowest watermark among the inflight messages and BTreeSet provides O(1) + /// time complexity for getting the lowest watermark, even though insertion and deletion are + /// O(log n). If we use map or hashset, our lowest watermark fetch call would be O(n) even + /// though insertion and deletion are O(1). We do almost same amount insertion, deletion and + /// getting the lowest watermark so BTreeSet is the best choice. + offset_set: HashMap>, +} + +impl ISBWatermarkActor { + fn new(fetcher: ISBWatermarkFetcher, publisher: ISBWatermarkPublisher) -> Self { + Self { + fetcher, + publisher, + offset_set: HashMap::new(), + } + } + + /// run listens for messages and handles them + async fn run(mut self, mut receiver: Receiver) { + while let Some(message) = receiver.recv().await { + if let Err(e) = self.handle_message(message).await { + error!("error handling message: {:?}", e); + } + } + } + + /// handle_message handles the incoming actor message. + async fn handle_message(&mut self, message: ISBWaterMarkActorMessage) -> Result<()> { + match message { + // fetches the watermark for the given offset + ISBWaterMarkActorMessage::FetchWatermark { offset, oneshot_tx } => { + let watermark = self + .fetcher + .fetch_watermark(offset.offset, offset.partition_idx) + .await?; + + oneshot_tx + .send(Ok(watermark)) + .map_err(|_| Error::Watermark("failed to send response".to_string()))?; + } + + // gets the lowest watermark among the inflight requests and publishes the watermark + // for the offset and stream + ISBWaterMarkActorMessage::PublishWatermark { offset, stream } => { + let min_wm = self + .get_lowest_watermark() + .unwrap_or(Watermark::from_timestamp_millis(-1).unwrap()); + + self.publisher + .publish_watermark(stream, offset.offset, min_wm.timestamp_millis()) + .await?; + } + + // removes the offset from the tracked offsets + ISBWaterMarkActorMessage::RemoveOffset(offset) => { + self.remove_offset(offset.partition_idx, offset.offset)?; + } + + // inserts the offset to the tracked offsets + ISBWaterMarkActorMessage::InsertOffset { offset, watermark } => { + self.insert_offset(offset.partition_idx, offset.offset, watermark); + } + } + + Ok(()) + } + + /// insert the offset and watermark for inflight requests set + fn insert_offset(&mut self, partition_id: u16, offset: i64, watermark: Watermark) { + let set = self.offset_set.entry(partition_id).or_default(); + set.insert(OffsetWatermark { offset, watermark }); + } + + /// removes the offset from the inflight offsets set of the partition + fn remove_offset(&mut self, partition_id: u16, offset: i64) -> Result<()> { + if let Some(set) = self.offset_set.get_mut(&partition_id) { + if let Some(&OffsetWatermark { watermark, .. }) = + set.iter().find(|ow| ow.offset == offset) + { + set.remove(&OffsetWatermark { offset, watermark }); + } + } + Ok(()) + } + + /// gets the lowest watermark among all the inflight requests + fn get_lowest_watermark(&self) -> Option { + self.offset_set + .values() + .filter_map(|set| set.iter().next().map(|ow| ow.watermark)) + .min() + } +} + +/// Handle to interact with the EdgeWatermarkActor, exposes methods to fetch and publish watermarks +/// for the edges +#[derive(Clone)] +pub(crate) struct ISBWatermarkHandle { + sender: tokio::sync::mpsc::Sender, +} + +impl ISBWatermarkHandle { + /// new creates a new [ISBWatermarkHandle]. + pub(crate) async fn new( + vertex_name: &'static str, + vertex_replica: u16, + js_context: async_nats::jetstream::Context, + config: &EdgeWatermarkConfig, + ) -> Result { + let (sender, receiver) = tokio::sync::mpsc::channel(100); + + // create a processor manager map (from_vertex -> ProcessorManager) + let mut processor_managers = HashMap::new(); + for from_bucket_config in &config.from_vertex_config { + let processor_manager = + ProcessorManager::new(js_context.clone(), from_bucket_config).await?; + processor_managers.insert(from_bucket_config.vertex, processor_manager); + } + let fetcher = + ISBWatermarkFetcher::new(processor_managers, &config.from_vertex_config).await?; + + let processor_name = format!("{}-{}", vertex_name, vertex_replica); + let publisher = ISBWatermarkPublisher::new( + processor_name, + js_context.clone(), + &config.to_vertex_config, + ) + .await?; + + let actor = ISBWatermarkActor::new(fetcher, publisher); + tokio::spawn(async move { actor.run(receiver).await }); + Ok(Self { sender }) + } + + /// Fetches the watermark for the given offset. + pub(crate) async fn fetch_watermark(&self, offset: Offset) -> Result { + if let Offset::Int(offset) = offset { + let (oneshot_tx, oneshot_rx) = tokio::sync::oneshot::channel(); + self.sender + .send(ISBWaterMarkActorMessage::FetchWatermark { offset, oneshot_tx }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + + oneshot_rx + .await + .map_err(|_| Error::Watermark("failed to receive response".to_string()))? + } else { + Err(Error::Watermark("invalid offset type".to_string())) + } + } + + /// publish_watermark publishes the watermark for the given stream and offset. + pub(crate) async fn publish_watermark(&self, stream: Stream, offset: Offset) -> Result<()> { + if let Offset::Int(offset) = offset { + self.sender + .send(ISBWaterMarkActorMessage::PublishWatermark { offset, stream }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + Ok(()) + } else { + Err(Error::Watermark("invalid offset type".to_string())) + } + } + + /// remove_offset removes the offset from the tracked offsets. + pub(crate) async fn remove_offset(&self, offset: Offset) -> Result<()> { + if let Offset::Int(offset) = offset { + self.sender + .send(ISBWaterMarkActorMessage::RemoveOffset(offset)) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + Ok(()) + } else { + Err(Error::Watermark("invalid offset type".to_string())) + } + } + + /// insert_offset inserts the offset to the tracked offsets. + pub(crate) async fn insert_offset( + &self, + offset: Offset, + watermark: Option, + ) -> Result<()> { + if let Offset::Int(offset) = offset { + self.sender + .send(ISBWaterMarkActorMessage::InsertOffset { + offset, + watermark: watermark.unwrap_or( + Watermark::from_timestamp_millis(-1).expect("failed to parse time"), + ), + }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + Ok(()) + } else { + Err(Error::Watermark("invalid offset type".to_string())) + } + } +} + +#[cfg(test)] +mod tests { + use async_nats::jetstream; + use async_nats::jetstream::kv::Config; + use tokio::time::sleep; + + use super::*; + use crate::config::pipeline::isb::Stream; + use crate::config::pipeline::watermark::BucketConfig; + use crate::message::IntOffset; + use crate::watermark::wmb::WMB; + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_isb_watermark_handle_publish_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_isb_publish_watermark_OT"; + let hb_bucket_name = "test_isb_publish_watermark_PROCESSORS"; + let to_ot_bucket_name = "test_isb_publish_watermark_to_OT"; + let to_hb_bucket_name = "test_isb_publish_watermark_to_PROCESSORS"; + + let vertex_name = "test-vertex"; + + let from_bucket_config = BucketConfig { + vertex: "from_vertex", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + let to_bucket_config = BucketConfig { + vertex: "to_vertex", + partitions: 1, + ot_bucket: to_ot_bucket_name, + hb_bucket: to_hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: to_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: to_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let edge_config = EdgeWatermarkConfig { + from_vertex_config: vec![from_bucket_config.clone()], + to_vertex_config: vec![to_bucket_config.clone()], + }; + + let handle = ISBWatermarkHandle::new(vertex_name, 0, js_context.clone(), &edge_config) + .await + .expect("Failed to create ISBWatermarkHandle"); + + handle + .insert_offset( + Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + }), + Some(Watermark::from_timestamp_millis(100).unwrap()), + ) + .await + .expect("Failed to insert offset"); + + handle + .insert_offset( + Offset::Int(IntOffset { + offset: 2, + partition_idx: 0, + }), + Some(Watermark::from_timestamp_millis(200).unwrap()), + ) + .await + .expect("Failed to insert offset"); + + handle + .publish_watermark( + Stream { + name: "test_stream", + vertex: "from_vertex", + partition: 0, + }, + Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + }), + ) + .await + .expect("Failed to publish watermark"); + + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let mut wmb_found = true; + for _ in 0..10 { + let wmb = ot_bucket + .get("test-vertex-0") + .await + .expect("Failed to get wmb"); + + if let Some(wmb) = wmb { + let wmb: WMB = wmb.try_into().unwrap(); + assert_eq!(wmb.watermark, 100); + wmb_found = true; + } + sleep(std::time::Duration::from_millis(10)).await; + } + + if !wmb_found { + panic!("WMB not found"); + } + + // remove the smaller offset and then publish watermark and see + handle + .remove_offset(Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + })) + .await + .expect("Failed to remove offset"); + + handle + .publish_watermark( + Stream { + name: "test_stream", + vertex: "from_vertex", + partition: 0, + }, + Offset::Int(IntOffset { + offset: 2, + partition_idx: 0, + }), + ) + .await + .unwrap(); + + let mut wmb_found = true; + for _ in 0..10 { + let wmb = ot_bucket + .get("test-vertex-0") + .await + .expect("Failed to get wmb"); + + if let Some(wmb) = wmb { + let wmb: WMB = wmb.try_into().unwrap(); + assert_eq!(wmb.watermark, 200); + wmb_found = true; + } + sleep(std::time::Duration::from_millis(10)).await; + } + + if !wmb_found { + panic!("WMB not found"); + } + + // delete the stores + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(to_hb_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_fetch_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_fetch_watermark_OT"; + let hb_bucket_name = "test_fetch_watermark_PROCESSORS"; + + let vertex_name = "test-vertex"; + + let from_bucket_config = BucketConfig { + vertex: "from_vertex", + partitions: 1, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let edge_config = EdgeWatermarkConfig { + from_vertex_config: vec![from_bucket_config.clone()], + to_vertex_config: vec![from_bucket_config.clone()], + }; + + let handle = ISBWatermarkHandle::new(vertex_name, 0, js_context.clone(), &edge_config) + .await + .expect("Failed to create ISBWatermarkHandle"); + + let mut fetched_watermark = -1; + // publish watermark and try fetching to see if something is getting published + for i in 0..10 { + let offset = Offset::Int(IntOffset { + offset: i, + partition_idx: 0, + }); + + handle + .insert_offset( + offset.clone(), + Some(Watermark::from_timestamp_millis(i * 100).unwrap()), + ) + .await + .expect("Failed to insert offset"); + + handle + .publish_watermark( + Stream { + name: "test_stream", + vertex: "from_vertex", + partition: 0, + }, + Offset::Int(IntOffset { + offset: i, + partition_idx: 0, + }), + ) + .await + .expect("Failed to publish watermark"); + + let watermark = handle + .fetch_watermark(Offset::Int(IntOffset { + offset: 3, + partition_idx: 0, + })) + .await + .expect("Failed to fetch watermark"); + + if watermark.timestamp_millis() != -1 { + fetched_watermark = watermark.timestamp_millis(); + break; + } + sleep(std::time::Duration::from_millis(10)).await; + handle + .remove_offset(offset.clone()) + .await + .expect("Failed to insert offset"); + } + + assert_ne!(fetched_watermark, -1); + + // delete the stores + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + } +} diff --git a/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs b/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs new file mode 100644 index 0000000000..eea2e5ded7 --- /dev/null +++ b/rust/numaflow-core/src/watermark/isb/wm_fetcher.rs @@ -0,0 +1,855 @@ +//! Fetches watermark for the messages read from the ISB. It keeps track of the previous vertices +//! (could be more than one in case of join vertex) processors and their published watermarks +//! for each partition and fetches the watermark for the given offset and partition by iterating over +//! all the processor managers and getting the smallest watermark. It also deletes the processors that +//! are inactive. Since the vertex could be reading from multiple partitions, it keeps track of the +//! last fetched watermark per partition and returns the smallest watermark among all the last fetched +//! watermarks across the partitions this is to make sure the watermark is min across all the incoming +//! partitions. +use std::collections::HashMap; + +use crate::config::pipeline::watermark::BucketConfig; +use crate::error::{Error, Result}; +use crate::watermark::processor::manager::ProcessorManager; +use crate::watermark::wmb::Watermark; + +/// ISBWatermarkFetcher is the watermark fetcher for the incoming edges. +pub(crate) struct ISBWatermarkFetcher { + /// A map of vertex to its ProcessorManager. Each key represents the incoming vertex, it will + /// be > 1 only during JOIN. + processor_managers: HashMap<&'static str, ProcessorManager>, + /// A map of vertex to its last processed watermark for each partition. Index[0] will be 0th + /// partition, and so forth. + last_processed_wm: HashMap<&'static str, Vec>, +} + +impl ISBWatermarkFetcher { + /// Creates a new ISBWatermarkFetcher. + pub(crate) async fn new( + processor_managers: HashMap<&'static str, ProcessorManager>, + bucket_configs: &[BucketConfig], + ) -> Result { + let mut last_processed_wm = HashMap::new(); + + // Create a ProcessorManager for each edge. + for config in bucket_configs { + let processed_wm = vec![-1; config.partitions as usize]; + last_processed_wm.insert(config.vertex, processed_wm); + } + + Ok(ISBWatermarkFetcher { + processor_managers, + last_processed_wm, + }) + } + + /// Fetches the watermark for the given offset and partition. + pub(crate) async fn fetch_watermark( + &mut self, + offset: i64, + partition_idx: u16, + ) -> Result { + // Iterate over all the processor managers and get the smallest watermark. (join case) + for (edge, processor_manager) in self.processor_managers.iter() { + let mut epoch = i64::MAX; + let mut processors_to_delete = Vec::new(); + + // iterate over all the processors and get the smallest watermark + processor_manager + .processors + .read() + .expect("failed to acquire lock") + .iter() + .for_each(|(name, processor)| { + // we only need to consider the timeline for the requested partition + if let Some(timeline) = processor.timelines.get(partition_idx as usize) { + let t = timeline.get_event_time(offset); + if t < epoch { + epoch = t; + } + } + + // if the pod is not active and the head offset of all the timelines is less than the input offset, delete + // the processor (this means we are processing data later than what the stale processor has processed) + if processor.is_deleted() { + // headOffset is used to check whether this pod can be deleted (e.g., dead pod) + let head_offset = processor + .timelines + .iter() + .map(|timeline| timeline.get_head_offset()) + .max() + .unwrap_or(-1); + + if offset > head_offset { + processors_to_delete.push(name.clone()); + } + } + }); + + // delete the processors that are inactive + for name in processors_to_delete { + processor_manager.delete_processor(&name); + } + + // if the epoch is not i64::MAX, update the last processed watermark for this particular edge and the partition + // while fetching watermark we need to consider the smallest last processed watermark among all the partitions + if epoch != i64::MAX { + self.last_processed_wm + .get_mut(edge) + .unwrap_or_else(|| panic!("invalid vertex {}", edge)) + [partition_idx as usize] = epoch; + } + } + + // now we computed and updated for this partition, we just need to compare across partitions. + self.get_watermark() + } + + /// Fetches the latest idle WMB with the smallest watermark for the given partition + /// Only returns one if all Publishers are idle and if it's the smallest one of any partitions + #[allow(dead_code)] + pub(crate) async fn fetch_head_idle_watermark(&mut self, partition_idx: u16) -> Result { + let mut min_wm = i64::MAX; + for (edge, processor_manager) in self.processor_managers.iter() { + let mut epoch = i64::MAX; + for processor in processor_manager + .processors + .read() + .expect("failed to acquire lock") + .values() + { + // if the processor is not active, skip + if !processor.is_active() { + continue; + } + + // retrieve the head watermark of the partition, we only care about the head watermark + // because by looking at the head wmb we will know whether the processor is idle or not + let head_wmb = processor + .timelines + .get(partition_idx as usize) + .ok_or(Error::Watermark("Partition not found".to_string()))? + .get_head_wmb(); + + if let Some(wmb) = head_wmb { + // if the processor is not idle, return early + if !wmb.idle { + return Ok(-1); + } + // consider the smallest watermark among all the partitions + if wmb.watermark < epoch { + epoch = wmb.watermark; + } + } + } + + if epoch < min_wm { + min_wm = epoch; + } + + if epoch != i64::MAX { + // update the last processed watermark for this particular edge and the partition + self.last_processed_wm.get_mut(edge).expect("invalid edge") + [partition_idx as usize] = epoch; + } + } + + if min_wm == i64::MAX { + return Ok(-1); + } + + Ok(min_wm) + } + + /// returns the smallest last processed watermark among all the partitions + fn get_watermark(&self) -> Result { + let mut min_wm = i64::MAX; + for wm in self.last_processed_wm.values() { + for &w in wm { + if min_wm > w { + min_wm = w; + } + } + } + + if min_wm == i64::MAX { + return Ok(Watermark::from_timestamp_millis(-1).expect("failed to parse time")); + } + Ok(Watermark::from_timestamp_millis(min_wm).expect("failed to parse time")) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use bytes::Bytes; + use std::sync::RwLock; + + use super::*; + use crate::watermark::processor::manager::{Processor, Status}; + use crate::watermark::processor::timeline::OffsetTimeline; + use crate::watermark::wmb::WMB; + + #[tokio::test] + async fn test_fetch_watermark_single_edge_single_processor_single_partition() { + // Create a ProcessorManager with a single Processor and a single OffsetTimeline + let processor_name = Bytes::from("processor1"); + let mut processor = Processor::new(processor_name.clone(), Status::Active, 1); + let timeline = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 2, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 300, + offset: 3, + idle: false, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); + + processor.timelines[0] = timeline; + + let mut processors = HashMap::new(); + processors.insert(processor_name.clone(), processor); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_watermark and verify the result + let watermark = fetcher.fetch_watermark(2, 0).await.unwrap(); + assert_eq!(watermark.timestamp_millis(), 100); + } + + #[tokio::test] + async fn test_fetch_watermark_single_edge_multi_processor_single_partition() { + // Create ProcessorManager with multiple Processors and different OffsetTimelines + let processor_name1 = Bytes::from("processor1"); + let processor_name2 = Bytes::from("processor2"); + let processor_name3 = Bytes::from("processor3"); + + let mut processor1 = Processor::new(processor_name1.clone(), Status::Active, 1); + let mut processor2 = Processor::new(processor_name2.clone(), Status::Active, 1); + let mut processor3 = Processor::new(processor_name3.clone(), Status::Active, 1); + + let timeline1 = OffsetTimeline::new(10); + let timeline2 = OffsetTimeline::new(10); + let timeline3 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries with unique and mixed offsets + let wmbs1 = vec![ + WMB { + watermark: 100, + offset: 5, + idle: false, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: false, + partition: 0, + }, + WMB { + watermark: 200, + offset: 15, + idle: false, + partition: 0, + }, + WMB { + watermark: 250, + offset: 20, + idle: false, + partition: 0, + }, + ]; + let wmbs2 = vec![ + WMB { + watermark: 110, + offset: 3, + idle: false, + partition: 0, + }, + WMB { + watermark: 160, + offset: 8, + idle: false, + partition: 0, + }, + WMB { + watermark: 210, + offset: 13, + idle: false, + partition: 0, + }, + WMB { + watermark: 260, + offset: 18, + idle: false, + partition: 0, + }, + ]; + let wmbs3 = vec![ + WMB { + watermark: 120, + offset: 2, + idle: false, + partition: 0, + }, + WMB { + watermark: 170, + offset: 7, + idle: false, + partition: 0, + }, + WMB { + watermark: 220, + offset: 12, + idle: false, + partition: 0, + }, + WMB { + watermark: 270, + offset: 17, + idle: false, + partition: 0, + }, + ]; + + for wmb in wmbs1 { + timeline1.put(wmb); + } + for wmb in wmbs2 { + timeline2.put(wmb); + } + for wmb in wmbs3 { + timeline3.put(wmb); + } + + processor1.timelines[0] = timeline1; + processor2.timelines[0] = timeline2; + processor3.timelines[0] = timeline3; + + let mut processors = HashMap::new(); + processors.insert(processor_name1.clone(), processor1); + processors.insert(processor_name2.clone(), processor2); + processors.insert(processor_name3.clone(), processor3); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_watermark and verify the result + let watermark = fetcher.fetch_watermark(12, 0).await.unwrap(); + assert_eq!(watermark.timestamp_millis(), 150); + } + + #[tokio::test] + async fn test_fetch_watermark_single_edge_multi_processor_multi_partition() { + // Create ProcessorManager with multiple Processors and different OffsetTimelines + let processor_name1 = Bytes::from("processor1"); + let processor_name2 = Bytes::from("processor2"); + let processor_name3 = Bytes::from("processor3"); + + let mut processor1 = Processor::new(processor_name1.clone(), Status::Active, 2); + let mut processor2 = Processor::new(processor_name2.clone(), Status::Active, 2); + let mut processor3 = Processor::new(processor_name3.clone(), Status::Active, 2); + + let timeline1_p0 = OffsetTimeline::new(10); + let timeline1_p1 = OffsetTimeline::new(10); + let timeline2_p0 = OffsetTimeline::new(10); + let timeline2_p1 = OffsetTimeline::new(10); + let timeline3_p0 = OffsetTimeline::new(10); + let timeline3_p1 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries with unique and mixed offsets + let wmbs1_p0 = vec![ + WMB { + watermark: 100, + offset: 6, + idle: false, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: false, + partition: 0, + }, + WMB { + watermark: 200, + offset: 15, + idle: false, + partition: 0, + }, + WMB { + watermark: 250, + offset: 20, + idle: false, + partition: 0, + }, + ]; + let wmbs1_p1 = vec![ + WMB { + watermark: 110, + offset: 25, + idle: false, + partition: 1, + }, + WMB { + watermark: 160, + offset: 30, + idle: false, + partition: 1, + }, + WMB { + watermark: 210, + offset: 35, + idle: false, + partition: 1, + }, + WMB { + watermark: 260, + offset: 40, + idle: false, + partition: 1, + }, + ]; + let wmbs2_p0 = vec![ + WMB { + watermark: 120, + offset: 3, + idle: false, + partition: 0, + }, + WMB { + watermark: 170, + offset: 8, + idle: false, + partition: 0, + }, + WMB { + watermark: 220, + offset: 13, + idle: false, + partition: 0, + }, + WMB { + watermark: 270, + offset: 18, + idle: false, + partition: 0, + }, + ]; + let wmbs2_p1 = vec![ + WMB { + watermark: 130, + offset: 23, + idle: false, + partition: 1, + }, + WMB { + watermark: 180, + offset: 28, + idle: false, + partition: 1, + }, + WMB { + watermark: 230, + offset: 33, + idle: false, + partition: 1, + }, + WMB { + watermark: 280, + offset: 38, + idle: false, + partition: 1, + }, + ]; + let wmbs3_p0 = vec![ + WMB { + watermark: 140, + offset: 2, + idle: false, + partition: 0, + }, + WMB { + watermark: 190, + offset: 7, + idle: false, + partition: 0, + }, + WMB { + watermark: 240, + offset: 12, + idle: false, + partition: 0, + }, + WMB { + watermark: 290, + offset: 17, + idle: false, + partition: 0, + }, + ]; + let wmbs3_p1 = vec![ + WMB { + watermark: 150, + offset: 22, + idle: false, + partition: 1, + }, + WMB { + watermark: 200, + offset: 27, + idle: false, + partition: 1, + }, + WMB { + watermark: 250, + offset: 32, + idle: false, + partition: 1, + }, + WMB { + watermark: 300, + offset: 37, + idle: false, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0 { + timeline1_p0.put(wmb); + } + for wmb in wmbs1_p1 { + timeline1_p1.put(wmb); + } + for wmb in wmbs2_p0 { + timeline2_p0.put(wmb); + } + for wmb in wmbs2_p1 { + timeline2_p1.put(wmb); + } + for wmb in wmbs3_p0 { + timeline3_p0.put(wmb); + } + for wmb in wmbs3_p1 { + timeline3_p1.put(wmb); + } + + processor1.timelines[0] = timeline1_p0; + processor1.timelines[1] = timeline1_p1; + processor2.timelines[0] = timeline2_p0; + processor2.timelines[1] = timeline2_p1; + processor3.timelines[0] = timeline3_p0; + processor3.timelines[1] = timeline3_p1; + + let mut processors = HashMap::new(); + processors.insert(processor_name1.clone(), processor1); + processors.insert(processor_name2.clone(), processor2); + processors.insert(processor_name3.clone(), processor3); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("from_vtx", processor_manager); + + let bucket_config = BucketConfig { + vertex: "from_vtx", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 2, + }; + + let mut fetcher = ISBWatermarkFetcher::new(processor_managers, &[bucket_config]) + .await + .unwrap(); + + // Invoke fetch_watermark and verify the result for partition 0, first fetch will be -1 because we have not fetched for other + // partition (we consider min across the last fetched watermark) + let watermark_p0 = fetcher.fetch_watermark(12, 0).await.unwrap(); + assert_eq!(watermark_p0.timestamp_millis(), -1); + + // Invoke fetch_watermark and verify the result for partition 1 (we consider min across the last fetch wm for all partitions) + let watermark_p1 = fetcher.fetch_watermark(32, 1).await.unwrap(); + assert_eq!(watermark_p1.timestamp_millis(), 150); + } + + #[tokio::test] + async fn test_fetch_watermark_two_edges_multi_processor_multi_partition() { + // Create ProcessorManagers with multiple Processors and different OffsetTimelines for edge1 + let processor_name1_edge1 = Bytes::from("processor1_edge1"); + let processor_name2_edge1 = Bytes::from("processor2_edge1"); + + let mut processor1_edge1 = Processor::new(processor_name1_edge1.clone(), Status::Active, 2); + let mut processor2_edge1 = Processor::new(processor_name2_edge1.clone(), Status::Active, 2); + + let timeline1_p0_edge1 = OffsetTimeline::new(10); + let timeline1_p1_edge1 = OffsetTimeline::new(10); + let timeline2_p0_edge1 = OffsetTimeline::new(10); + let timeline2_p1_edge1 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries with unique and mixed offsets for edge1 + let wmbs1_p0_edge1 = vec![ + WMB { + watermark: 100, + offset: 6, + idle: false, + partition: 0, + }, + WMB { + watermark: 150, + offset: 10, + idle: false, + partition: 0, + }, + ]; + let wmbs1_p1_edge1 = vec![ + WMB { + watermark: 110, + offset: 25, + idle: false, + partition: 1, + }, + WMB { + watermark: 160, + offset: 30, + idle: false, + partition: 1, + }, + ]; + let wmbs2_p0_edge1 = vec![ + WMB { + watermark: 120, + offset: 3, + idle: false, + partition: 0, + }, + WMB { + watermark: 170, + offset: 8, + idle: false, + partition: 0, + }, + ]; + let wmbs2_p1_edge1 = vec![ + WMB { + watermark: 130, + offset: 23, + idle: false, + partition: 1, + }, + WMB { + watermark: 180, + offset: 28, + idle: false, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0_edge1 { + timeline1_p0_edge1.put(wmb); + } + for wmb in wmbs1_p1_edge1 { + timeline1_p1_edge1.put(wmb); + } + for wmb in wmbs2_p0_edge1 { + timeline2_p0_edge1.put(wmb); + } + for wmb in wmbs2_p1_edge1 { + timeline2_p1_edge1.put(wmb); + } + + processor1_edge1.timelines[0] = timeline1_p0_edge1; + processor1_edge1.timelines[1] = timeline1_p1_edge1; + processor2_edge1.timelines[0] = timeline2_p0_edge1; + processor2_edge1.timelines[1] = timeline2_p1_edge1; + + let mut processors_edge1 = HashMap::new(); + processors_edge1.insert(processor_name1_edge1.clone(), processor1_edge1); + processors_edge1.insert(processor_name2_edge1.clone(), processor2_edge1); + + let processor_manager_edge1 = ProcessorManager { + processors: Arc::new(RwLock::new(processors_edge1)), + handles: vec![], + }; + + // Create ProcessorManagers with multiple Processors and different OffsetTimelines for edge2 + let processor_name1_edge2 = Bytes::from("processor1_edge2"); + let processor_name2_edge2 = Bytes::from("processor2_edge2"); + + let mut processor1_edge2 = Processor::new(processor_name1_edge2.clone(), Status::Active, 2); + let mut processor2_edge2 = Processor::new(processor_name2_edge2.clone(), Status::Active, 2); + + let timeline1_p0_edge2 = OffsetTimeline::new(10); + let timeline1_p1_edge2 = OffsetTimeline::new(10); + let timeline2_p0_edge2 = OffsetTimeline::new(10); + let timeline2_p1_edge2 = OffsetTimeline::new(10); + + // Populate the OffsetTimelines with sorted WMB entries with unique and mixed offsets for edge2 + let wmbs1_p0_edge2 = vec![ + WMB { + watermark: 140, + offset: 2, + idle: false, + partition: 0, + }, + WMB { + watermark: 190, + offset: 7, + idle: false, + partition: 0, + }, + ]; + let wmbs1_p1_edge2 = vec![ + WMB { + watermark: 150, + offset: 22, + idle: false, + partition: 1, + }, + WMB { + watermark: 200, + offset: 27, + idle: false, + partition: 1, + }, + ]; + let wmbs2_p0_edge2 = vec![ + WMB { + watermark: 160, + offset: 4, + idle: false, + partition: 0, + }, + WMB { + watermark: 210, + offset: 9, + idle: false, + partition: 0, + }, + ]; + let wmbs2_p1_edge2 = vec![ + WMB { + watermark: 170, + offset: 24, + idle: false, + partition: 1, + }, + WMB { + watermark: 220, + offset: 29, + idle: false, + partition: 1, + }, + ]; + + for wmb in wmbs1_p0_edge2 { + timeline1_p0_edge2.put(wmb); + } + for wmb in wmbs1_p1_edge2 { + timeline1_p1_edge2.put(wmb); + } + for wmb in wmbs2_p0_edge2 { + timeline2_p0_edge2.put(wmb); + } + for wmb in wmbs2_p1_edge2 { + timeline2_p1_edge2.put(wmb); + } + + processor1_edge2.timelines[0] = timeline1_p0_edge2; + processor1_edge2.timelines[1] = timeline1_p1_edge2; + processor2_edge2.timelines[0] = timeline2_p0_edge2; + processor2_edge2.timelines[1] = timeline2_p1_edge2; + + let mut processors_edge2 = HashMap::new(); + processors_edge2.insert(processor_name1_edge2.clone(), processor1_edge2); + processors_edge2.insert(processor_name2_edge2.clone(), processor2_edge2); + + let processor_manager_edge2 = ProcessorManager { + processors: Arc::new(RwLock::new(processors_edge2)), + handles: vec![], + }; + + let mut processor_managers = HashMap::new(); + processor_managers.insert("edge1", processor_manager_edge1); + processor_managers.insert("edge2", processor_manager_edge2); + + let bucket_config1 = BucketConfig { + vertex: "edge1", + ot_bucket: "ot_bucket1", + hb_bucket: "hb_bucket1", + partitions: 2, + }; + let bucket_config2 = BucketConfig { + vertex: "edge2", + ot_bucket: "ot_bucket2", + hb_bucket: "hb_bucket2", + partitions: 2, + }; + + let mut fetcher = + ISBWatermarkFetcher::new(processor_managers, &[bucket_config1, bucket_config2]) + .await + .unwrap(); + + // Invoke fetch_watermark and verify the result for partition 0 + let watermark_p0 = fetcher.fetch_watermark(12, 0).await.unwrap(); + assert_eq!(watermark_p0.timestamp_millis(), -1); + + // Invoke fetch_watermark and verify the result for partition 1 + let watermark_p1 = fetcher.fetch_watermark(32, 1).await.unwrap(); + assert_eq!(watermark_p1.timestamp_millis(), 150); + } +} diff --git a/rust/numaflow-core/src/watermark/isb/wm_publisher.rs b/rust/numaflow-core/src/watermark/isb/wm_publisher.rs new file mode 100644 index 0000000000..6b5d3e85e5 --- /dev/null +++ b/rust/numaflow-core/src/watermark/isb/wm_publisher.rs @@ -0,0 +1,442 @@ +//! Publishes watermark of the messages written to ISB. Each publisher is mapped to a processing entity +//! which could be a pod or a partition, it also creates a background task to publish heartbeats for the +//! downstream vertices, to indicate the liveliness of the processor. It publishes watermark to the +//! appropriate OT bucket based on stream information provided. It makes sure we always publish m +//! increasing watermark. +use std::collections::HashMap; +use std::time::UNIX_EPOCH; +use std::time::{Duration, SystemTime}; + +use bytes::BytesMut; +use prost::Message; +use tracing::info; + +use crate::config::pipeline::isb::Stream; +use crate::config::pipeline::watermark::BucketConfig; +use crate::error::{Error, Result}; +use crate::watermark::wmb::WMB; + +/// Interval at which the pod sends heartbeats. +const DEFAULT_POD_HEARTBEAT_INTERVAL: u16 = 5; + +/// LastPublishedState is the state of the last published watermark and offset +/// for a partition. +#[derive(Clone, Debug)] +struct LastPublishedState { + offset: i64, + watermark: i64, +} + +/// ISBWatermarkPublisher is the watermark publisher for the outgoing edges. +pub(crate) struct ISBWatermarkPublisher { + /// name of the processor(node) that is publishing the watermark. + processor_name: String, + /// handle to the heartbeat publishing task. + hb_handle: tokio::task::JoinHandle<()>, + /// last published watermark for each vertex and partition. + last_published_wm: HashMap<&'static str, Vec>, + /// map of vertex to its ot bucket. + ot_buckets: HashMap<&'static str, async_nats::jetstream::kv::Store>, +} + +impl Drop for ISBWatermarkPublisher { + fn drop(&mut self) { + self.hb_handle.abort(); + } +} + +impl ISBWatermarkPublisher { + /// Creates a new ISBWatermarkPublisher. + pub(crate) async fn new( + processor_name: String, + js_context: async_nats::jetstream::Context, + bucket_configs: &[BucketConfig], + ) -> Result { + let mut ot_buckets = HashMap::new(); + let mut hb_buckets = Vec::with_capacity(bucket_configs.len()); + let mut last_published_wm = HashMap::new(); + + // create ot and hb buckets + for config in bucket_configs { + let js_context = js_context.clone(); + let ot_bucket = js_context + .get_key_value(config.ot_bucket) + .await + .map_err(|e| Error::Watermark(e.to_string()))?; + + let hb_bucket = js_context + .get_key_value(config.hb_bucket) + .await + .map_err(|e| Error::Watermark(e.to_string()))?; + + ot_buckets.insert(config.vertex, ot_bucket); + hb_buckets.push(hb_bucket); + last_published_wm.insert( + config.vertex, + vec![ + LastPublishedState { + offset: -1, + watermark: -1, + }; + config.partitions as usize + ], + ); + } + + // start publishing heartbeats + let hb_handle = tokio::spawn(Self::start_heartbeat(processor_name.clone(), hb_buckets)); + + Ok(ISBWatermarkPublisher { + processor_name, + hb_handle, + last_published_wm, + ot_buckets, + }) + } + + /// start_heartbeat starts publishing heartbeats to the hb buckets + async fn start_heartbeat( + processor_name: String, + hb_buckets: Vec, + ) { + let mut interval = + tokio::time::interval(Duration::from_secs(DEFAULT_POD_HEARTBEAT_INTERVAL as u64)); + info!(processor = ?processor_name, "Started publishing heartbeat"); + loop { + interval.tick().await; + let heartbeat = numaflow_pb::objects::watermark::Heartbeat { + heartbeat: SystemTime::now() + .duration_since(UNIX_EPOCH) + .expect("Failed to get duration since epoch") + .as_secs() as i64, + }; + + let mut bytes = BytesMut::new(); + heartbeat + .encode(&mut bytes) + .expect("Failed to encode heartbeat"); + + for hb_bucket in hb_buckets.iter() { + info!(heartbeat = ?heartbeat.heartbeat, processor = ?processor_name, + "Publishing heartbeat", + ); + hb_bucket + .put(processor_name.clone(), bytes.clone().freeze()) + .await + .expect("Failed to publish heartbeat"); + } + } + } + + /// publish_watermark publishes the watermark for the given offset and the stream. + pub(crate) async fn publish_watermark( + &mut self, + stream: Stream, + offset: i64, + watermark: i64, + ) -> Result<()> { + let last_published_wm_state = match self.last_published_wm.get_mut(stream.vertex) { + Some(wm) => wm, + None => return Err(Error::Watermark("Invalid vertex".to_string())), + }; + + // we can avoid publishing the watermark if it is <= the last published watermark (optimization) + let last_state = &last_published_wm_state[stream.partition as usize]; + if offset <= last_state.offset || watermark <= last_state.watermark { + return Ok(()); + } + + let ot_bucket = self.ot_buckets.get(stream.vertex).ok_or(Error::Watermark( + "Invalid vertex, no ot bucket found".to_string(), + ))?; + + let wmb_bytes: BytesMut = WMB { + idle: false, + offset, + watermark, + partition: stream.partition, + } + .try_into() + .map_err(|e| Error::Watermark(format!("{}", e)))?; + ot_bucket + .put(self.processor_name.clone(), wmb_bytes.freeze()) + .await + .map_err(|e| Error::Watermark(e.to_string()))?; + + // update the last published watermark state + last_published_wm_state[stream.partition as usize] = + LastPublishedState { offset, watermark }; + + Ok(()) + } +} + +#[cfg(test)] +mod tests { + use async_nats::jetstream; + use async_nats::jetstream::kv::Config; + + use crate::config::pipeline::isb::Stream; + use crate::config::pipeline::watermark::BucketConfig; + use crate::watermark::isb::wm_publisher::ISBWatermarkPublisher; + use crate::watermark::wmb::WMB; + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_isb_publisher_one_edge() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "isb_publisher_one_edge_OT"; + let hb_bucket_name = "isb_publisher_one_edge_PROCESSORS"; + + let bucket_configs = vec![BucketConfig { + vertex: "v1", + partitions: 2, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }]; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut publisher = ISBWatermarkPublisher::new( + "processor1".to_string(), + js_context.clone(), + &bucket_configs, + ) + .await + .expect("Failed to create publisher"); + + let stream_partition_0 = Stream { + name: "v1-0", + vertex: "v1", + partition: 0, + }; + + let stream_partition_1 = Stream { + name: "v1-1", + vertex: "v1", + partition: 1, + }; + + // Publish watermark for partition 0 + publisher + .publish_watermark(stream_partition_0.clone(), 1, 100) + .await + .expect("Failed to publish watermark"); + + let ot_bucket = js_context + .get_key_value("isb_publisher_one_edge_OT") + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("processor1") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 1); + assert_eq!(wmb.watermark, 100); + + // Try publishing a smaller watermark for the same partition, it should not be published + publisher + .publish_watermark(stream_partition_0.clone(), 0, 50) + .await + .expect("Failed to publish watermark"); + + let wmb = ot_bucket + .get("processor1") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 1); + assert_eq!(wmb.watermark, 100); + + // Publish a smaller watermark for a different partition, it should be published + publisher + .publish_watermark(stream_partition_1.clone(), 0, 50) + .await + .expect("Failed to publish watermark"); + + let wmb = ot_bucket + .get("processor1") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 0); + assert_eq!(wmb.watermark, 50); + + // delete the stores + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_isb_publisher_multi_edges() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name_v1 = "isb_publisher_multi_edges_v1_OT"; + let hb_bucket_name_v1 = "isb_publisher_multi_edges_v1_PROCESSORS"; + let ot_bucket_name_v2 = "isb_publisher_multi_edges_v2_OT"; + let hb_bucket_name_v2 = "isb_publisher_multi_edges_v2_PROCESSORS"; + + let bucket_configs = vec![ + BucketConfig { + vertex: "v1", + partitions: 1, + ot_bucket: ot_bucket_name_v1, + hb_bucket: hb_bucket_name_v1, + }, + BucketConfig { + vertex: "v2", + partitions: 1, + ot_bucket: ot_bucket_name_v2, + hb_bucket: hb_bucket_name_v2, + }, + ]; + + // create key value stores for v1 + js_context + .create_key_value(Config { + bucket: ot_bucket_name_v1.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name_v1.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + // create key value stores for v2 + js_context + .create_key_value(Config { + bucket: ot_bucket_name_v2.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: hb_bucket_name_v2.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut publisher = ISBWatermarkPublisher::new( + "processor1".to_string(), + js_context.clone(), + &bucket_configs, + ) + .await + .expect("Failed to create publisher"); + + let stream1 = Stream { + name: "v1-0", + vertex: "v1", + partition: 0, + }; + + let stream2 = Stream { + name: "v2-0", + vertex: "v2", + partition: 0, + }; + + publisher + .publish_watermark(stream1.clone(), 1, 100) + .await + .expect("Failed to publish watermark"); + + publisher + .publish_watermark(stream2.clone(), 1, 200) + .await + .expect("Failed to publish watermark"); + + let ot_bucket_v1 = js_context + .get_key_value(ot_bucket_name_v1) + .await + .expect("Failed to get ot bucket for v1"); + + let ot_bucket_v2 = js_context + .get_key_value(ot_bucket_name_v2) + .await + .expect("Failed to get ot bucket for v2"); + + let wmb_v1 = ot_bucket_v1 + .get("processor1") + .await + .expect("Failed to get wmb for v1"); + assert!(wmb_v1.is_some()); + + let wmb_v1: WMB = wmb_v1.unwrap().try_into().unwrap(); + assert_eq!(wmb_v1.offset, 1); + assert_eq!(wmb_v1.watermark, 100); + + let wmb_v2 = ot_bucket_v2 + .get("processor1") + .await + .expect("Failed to get wmb for v2"); + assert!(wmb_v2.is_some()); + + let wmb_v2: WMB = wmb_v2.unwrap().try_into().unwrap(); + assert_eq!(wmb_v2.offset, 1); + assert_eq!(wmb_v2.watermark, 200); + + // delete the stores + js_context + .delete_key_value(hb_bucket_name_v1.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name_v1.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(hb_bucket_name_v2.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name_v2.to_string()) + .await + .unwrap(); + } +} diff --git a/rust/numaflow-core/src/watermark/processor.rs b/rust/numaflow-core/src/watermark/processor.rs new file mode 100644 index 0000000000..f84b73ec91 --- /dev/null +++ b/rust/numaflow-core/src/watermark/processor.rs @@ -0,0 +1,11 @@ +//! Processor is an entity which can emit watermarks, it could be a pod or a partition. Each processor +//! will have a per-partition offset timeline to track the offsets and watermarks. At Vn vertex we will +//! have to track all the Vn-1 active processors to determine the watermark for the read offset. For +//! source the processor will be partition since watermark originates at source. Looking at all the +//! active processors and their timelines fetcher will determine the watermark for the read offset. + +/// manager for managing the processors (pod or partition). +pub(super) mod manager; + +/// offset timeline for tracking processor offsets and watermarks. +pub(super) mod timeline; diff --git a/rust/numaflow-core/src/watermark/processor/manager.rs b/rust/numaflow-core/src/watermark/processor/manager.rs new file mode 100644 index 0000000000..33f1a9ea0e --- /dev/null +++ b/rust/numaflow-core/src/watermark/processor/manager.rs @@ -0,0 +1,616 @@ +//! Manages the processors and their lifecycle. It will keep track of all the active processor by listening +//! to the heartbeat bucket and update their offset timelines by listening to the ot bucket. It will also +//! refresh the active processors if they are not active for a certain time. + +use std::collections::HashMap; +use std::fmt::{Debug, Formatter}; +use std::sync::Arc; +use std::time::Duration; +use std::time::SystemTime; + +use async_nats::jetstream::kv::Watch; +use backoff::retry::Retry; +use backoff::strategy::fixed; +use bytes::Bytes; +use futures::StreamExt; +use prost::Message as ProtoMessage; +use std::sync::RwLock; +use tracing::{debug, error, info, warn}; + +use crate::config::pipeline::watermark::BucketConfig; +use crate::error::{Error, Result}; +use crate::watermark::processor::timeline::OffsetTimeline; +use crate::watermark::wmb::WMB; + +const DEFAULT_PROCESSOR_REFRESH_RATE: u16 = 5; + +/// Status of a processor. +#[derive(Clone, Debug, PartialEq)] +pub(crate) enum Status { + InActive, + Active, + Deleted, +} + +/// Processor is the smallest unit of entity (from which we fetch data) that does inorder processing +/// or contains inorder data. It tracks OT for all the partitions of the from-buffer. +#[derive(Clone)] +pub(crate) struct Processor { + /// Name of the processor. + pub(crate) name: Bytes, + /// [Status] of the processor. + pub(crate) status: Status, + /// OffsetTimeline for each partition. + pub(crate) timelines: Vec, +} + +impl Debug for Processor { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + writeln!( + f, + "Processor: {:?}, Status: {:?}, Timelines: ", + self.name, self.status + )?; + for timeline in &self.timelines { + writeln!(f, "{:?}", timeline)?; + } + Ok(()) + } +} + +impl Processor { + pub(crate) fn new(name: Bytes, status: Status, partition_count: usize) -> Self { + let mut timelines = Vec::with_capacity(partition_count); + for _ in 0..partition_count { + timelines.push(OffsetTimeline::new(10)); + } + Processor { + name, + status, + timelines, + } + } + + /// Set the status of the processor. + pub(crate) fn set_status(&mut self, status: Status) { + self.status = status; + } + + /// Check if the processor is active. + pub(crate) fn is_active(&self) -> bool { + self.status == Status::Active + } + + /// Check if the processor is deleted. + pub(crate) fn is_deleted(&self) -> bool { + self.status == Status::Deleted + } +} + +/// processorManager manages the point of view of Vn-1 from the Vn vertex processor (or source processor). +/// The code is running on Vn vertex. It has the mapping of all the processors which in turn has all the +/// information about each processor timelines. +pub(crate) struct ProcessorManager { + /// Mapping of processor name to processor + pub(crate) processors: Arc>>, + /// Handles of ot listener, hb listener and processor refresher tasks + pub(crate) handles: Vec>, +} + +impl Debug for ProcessorManager { + fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result { + write!(f, "ProcessorManager: {:?}", self.processors) + } +} + +impl Drop for ProcessorManager { + fn drop(&mut self) { + for handle in self.handles.drain(..) { + handle.abort(); + } + } +} + +impl ProcessorManager { + /// Creates a new ProcessorManager. + pub(crate) async fn new( + js_context: async_nats::jetstream::Context, + bucket_config: &BucketConfig, + ) -> Result { + let ot_bucket = js_context + .get_key_value(bucket_config.ot_bucket) + .await + .map_err(|e| { + Error::Watermark(format!( + "Failed to get kv bucket {}: {}", + bucket_config.ot_bucket, e + )) + })?; + + let hb_bucket = js_context + .get_key_value(bucket_config.hb_bucket) + .await + .map_err(|e| { + Error::Watermark(format!( + "Failed to get kv bucket {}: {}", + bucket_config.hb_bucket, e + )) + })?; + + let processors = Arc::new(RwLock::new(HashMap::new())); + let heartbeats = Arc::new(RwLock::new(HashMap::new())); + + // start the ot watcher, to listen to the OT bucket and update the timelines + let ot_handle = tokio::spawn(Self::start_ot_watcher(ot_bucket, Arc::clone(&processors))); + + // start the hb watcher, to listen to the HB bucket and update the list of + // active processors + let hb_handle = tokio::spawn(Self::start_hb_watcher( + bucket_config.partitions, + hb_bucket, + Arc::clone(&heartbeats), + Arc::clone(&processors), + )); + + // start the processor refresher, to update the status of the processors + // based on the last heartbeat + let refresh_handle = tokio::spawn(Self::start_refreshing_processors( + DEFAULT_PROCESSOR_REFRESH_RATE, + Arc::clone(&processors), + Arc::clone(&heartbeats), + )); + + Ok(ProcessorManager { + processors, + handles: vec![ot_handle, hb_handle, refresh_handle], + }) + } + + /// Starts refreshing the processors status based on the last heartbeat, if the last heartbeat + /// is more than 10 times the refreshing rate, the processor is marked as deleted + async fn start_refreshing_processors( + refreshing_processors_rate: u16, + processors: Arc>>, + heartbeats: Arc>>, + ) { + let mut interval = + tokio::time::interval(Duration::from_secs(refreshing_processors_rate as u64)); + loop { + interval.tick().await; + let current_time = SystemTime::now() + .duration_since(SystemTime::UNIX_EPOCH) + .unwrap() + .as_secs() as i64; + + let heartbeats = heartbeats.read().expect("failed to acquire lock"); + let mut processors = processors.write().expect("failed to acquire lock"); + + heartbeats.iter().for_each(|(p_name, &p_time)| { + if let Some(p) = processors.get_mut(p_name) { + let status = match current_time - p_time { + diff if diff > 10 * refreshing_processors_rate as i64 => Status::Deleted, + diff if diff > refreshing_processors_rate as i64 => Status::InActive, + _ => Status::Active, + }; + p.set_status(status); + } + }); + } + } + + /// Starts the ot watcher, to listen to the OT bucket and update the timelines for the + /// processors. + async fn start_ot_watcher( + ot_bucket: async_nats::jetstream::kv::Store, + processors: Arc>>, + ) { + let mut ot_watcher = Self::create_watcher(ot_bucket.clone()).await; + + loop { + let Some(val) = ot_watcher.next().await else { + warn!("OT watcher stopped, recreating watcher"); + ot_watcher = Self::create_watcher(ot_bucket.clone()).await; + continue; + }; + + let kv = match val { + Ok(kv) => kv, + Err(e) => { + warn!(?e, "Failed to get next kv entry, recreating watcher"); + ot_watcher = Self::create_watcher(ot_bucket.clone()).await; + continue; + } + }; + + match kv.operation { + async_nats::jetstream::kv::Operation::Put => { + let processor_name = Bytes::from(kv.key); + let wmb: WMB = kv.value.try_into().expect("Failed to decode WMB"); + + info!(wmb = ?wmb, processor = ?processor_name, "Received wmb from watcher"); + if let Some(processor) = processors + .write() + .expect("failed to acquire lock") + .get_mut(&processor_name) + { + let timeline = &mut processor.timelines[wmb.partition as usize]; + timeline.put(wmb); + } else { + debug!(?processor_name, "Processor not found"); + } + } + async_nats::jetstream::kv::Operation::Delete + | async_nats::jetstream::kv::Operation::Purge => { + // we don't care about delete or purge operations + } + } + } + } + + /// Starts the hb watcher, to listen to the HB bucket, will also create the processor if it + /// doesn't exist. + async fn start_hb_watcher( + partition_count: u16, + hb_bucket: async_nats::jetstream::kv::Store, + heartbeats: Arc>>, + processors: Arc>>, + ) { + let mut hb_watcher = Self::create_watcher(hb_bucket.clone()).await; + + loop { + let Some(val) = hb_watcher.next().await else { + warn!("HB watcher stopped, recreating watcher"); + hb_watcher = Self::create_watcher(hb_bucket.clone()).await; + continue; + }; + + let kv = match val { + Ok(kv) => kv, + Err(e) => { + warn!(?e, "Failed to get next kv entry, recreating watcher"); + hb_watcher = Self::create_watcher(hb_bucket.clone()).await; + continue; + } + }; + + match kv.operation { + async_nats::jetstream::kv::Operation::Put => { + let processor_name = Bytes::from(kv.key); + let hb = numaflow_pb::objects::watermark::Heartbeat::decode(kv.value) + .expect("Failed to decode heartbeat") + .heartbeat; + heartbeats + .write() + .expect("failed to acquire lock") + .insert(processor_name.clone(), hb); + + info!(hb = ?hb, processor = ?processor_name, "Received heartbeat from watcher"); + // if the processor is not in the processors map, add it + // or if processor status is not active, set it to active + let mut processors = processors.write().expect("failed to acquire lock"); + if let Some(processor) = processors.get_mut(&processor_name) { + if !processor.is_active() { + processor.set_status(Status::Active); + } + } else { + info!(processor = ?processor_name, "Processor not found, adding it"); + let processor = Processor::new( + processor_name, + Status::Active, + partition_count as usize, + ); + processors.insert(processor.name.clone(), processor); + } + } + async_nats::jetstream::kv::Operation::Delete => { + let processor_name = Bytes::from(kv.key); + heartbeats + .write() + .expect("failed to acquire lock") + .remove(&processor_name); + + // update the processor status to deleted + if let Some(processor) = processors + .write() + .expect("failed to acquire lock") + .get_mut(&processor_name) + { + processor.set_status(Status::Deleted); + } + } + async_nats::jetstream::kv::Operation::Purge => { + heartbeats.write().expect("failed to acquire lock").clear(); + + // update the processor status to deleted + for (_, processor) in processors + .write() + .expect("failed to acquire lock") + .iter_mut() + { + processor.set_status(Status::Deleted); + } + } + } + } + } + + /// creates a watcher for the given bucket, will retry infinitely until it succeeds + async fn create_watcher(bucket: async_nats::jetstream::kv::Store) -> Watch { + const RECONNECT_INTERVAL: u64 = 1000; + + // infinite retry + let interval = fixed::Interval::from_millis(RECONNECT_INTERVAL).take(usize::MAX); + + Retry::retry( + interval, + || async { + match bucket.watch_all().await { + Ok(w) => Ok(w), + Err(e) => { + error!(?e, "Failed to create watcher"); + Err(Error::Watermark(format!("Failed to create watcher: {}", e))) + } + } + }, + |_: &Error| true, + ) + .await + .expect("Failed to create ot watcher") + } + + /// Delete a processor from the processors map + pub(crate) fn delete_processor(&self, processor_name: &Bytes) { + let mut processors = self.processors.write().expect("failed to acquire lock"); + processors.remove(processor_name); + } +} + +#[cfg(test)] +mod tests { + use async_nats::jetstream; + use async_nats::jetstream::kv::Config; + use async_nats::jetstream::kv::Store; + use async_nats::jetstream::Context; + use bytes::{Bytes, BytesMut}; + use prost::Message; + + use super::*; + + async fn setup_nats() -> Context { + let client = async_nats::connect("localhost:4222").await.unwrap(); + jetstream::new(client) + } + + async fn create_kv_bucket(js: &Context, bucket_name: &str) -> Store { + js.create_key_value(Config { + bucket: bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap() + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_processor_manager_tracks_heartbeats_and_wmbs() { + let js_context = setup_nats().await; + let ot_bucket = create_kv_bucket(&js_context, "ot_bucket").await; + let hb_bucket = create_kv_bucket(&js_context, "hb_bucket").await; + + let bucket_config = BucketConfig { + vertex: "test", + ot_bucket: "ot_bucket", + hb_bucket: "hb_bucket", + partitions: 1, + }; + + let processor_manager = ProcessorManager::new(js_context.clone(), &bucket_config) + .await + .unwrap(); + + let processor_name = Bytes::from("processor1"); + + // Spawn a task to keep publishing heartbeats + let hb_task = tokio::spawn(async move { + loop { + let heartbeat = numaflow_pb::objects::watermark::Heartbeat { heartbeat: 100 }; + let mut bytes = BytesMut::new(); + heartbeat + .encode(&mut bytes) + .expect("Failed to encode heartbeat"); + hb_bucket.put("processor1", bytes.freeze()).await.unwrap(); + tokio::time::sleep(Duration::from_millis(100)).await; + } + }); + + // Spawn a task to keep publishing WMBs + let ot_task = tokio::spawn(async move { + loop { + let wmb_bytes: BytesMut = WMB { + watermark: 200, + offset: 1, + idle: false, + partition: 0, + } + .try_into() + .unwrap(); + ot_bucket + .put("processor1", wmb_bytes.freeze()) + .await + .unwrap(); + tokio::time::sleep(Duration::from_millis(100)).await; + } + }); + + // Check every 10ms if the processor is added and the WMB is tracked + let start_time = tokio::time::Instant::now(); + loop { + tokio::time::sleep(Duration::from_millis(10)).await; + let processors = processor_manager + .processors + .read() + .expect("failed to acquire lock"); + if let Some(processor) = processors.get(&processor_name) { + if processor.status == Status::Active { + let timeline = &processor.timelines[0]; + if let Some(head_wmb) = timeline.get_head_wmb() { + if head_wmb.watermark == 200 && head_wmb.offset == 1 { + break; + } + } + } + } + if start_time.elapsed() > Duration::from_secs(1) { + panic!( + "Test failed: Processor was not added or WMB was not tracked within 1 second" + ); + } + } + + // Abort the tasks + hb_task.abort(); + ot_task.abort(); + + // delete the kv store + js_context.delete_key_value("ot_bucket").await.unwrap(); + js_context.delete_key_value("hb_bucket").await.unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_processor_manager_tracks_multiple_processors() { + let js_context = setup_nats().await; + let ot_bucket = + create_kv_bucket(&js_context, "test_processor_manager_multi_ot_bucket").await; + let hb_bucket = + create_kv_bucket(&js_context, "test_processor_manager_multi_hb_bucket").await; + + let bucket_config = BucketConfig { + vertex: "test", + ot_bucket: "test_processor_manager_multi_ot_bucket", + hb_bucket: "test_processor_manager_multi_hb_bucket", + partitions: 1, + }; + + let processor_manager = ProcessorManager::new(js_context.clone(), &bucket_config) + .await + .unwrap(); + + let processor_names = [ + Bytes::from("processor1"), + Bytes::from("processor2"), + Bytes::from("processor3"), + ]; + + // Spawn tasks to keep publishing heartbeats and WMBs for each processor + let hb_tasks: Vec<_> = processor_names + .iter() + .map(|processor_name| { + let hb_bucket = hb_bucket.clone(); + let processor_name = processor_name.clone(); + tokio::spawn(async move { + loop { + let heartbeat = + numaflow_pb::objects::watermark::Heartbeat { heartbeat: 100 }; + let mut bytes = BytesMut::new(); + heartbeat + .encode(&mut bytes) + .expect("Failed to encode heartbeat"); + hb_bucket + .put( + String::from_utf8(processor_name.to_vec()).unwrap(), + bytes.freeze(), + ) + .await + .unwrap(); + tokio::time::sleep(Duration::from_millis(100)).await; + } + }) + }) + .collect(); + + let ot_tasks: Vec<_> = processor_names + .iter() + .map(|processor_name| { + let ot_bucket = ot_bucket.clone(); + let processor_name = processor_name.clone(); + tokio::spawn(async move { + loop { + let wmb_bytes: BytesMut = WMB { + watermark: 200, + offset: 1, + idle: false, + partition: 0, + } + .try_into() + .unwrap(); + ot_bucket + .put( + String::from_utf8(processor_name.to_vec()).unwrap(), + wmb_bytes.freeze(), + ) + .await + .unwrap(); + tokio::time::sleep(Duration::from_millis(100)).await; + } + }) + }) + .collect(); + + // Check every 10ms if the processors are added and the WMBs are tracked + let start_time = tokio::time::Instant::now(); + loop { + tokio::time::sleep(Duration::from_millis(10)).await; + let processors = processor_manager + .processors + .read() + .expect("failed to acquire lock"); + + let futures: Vec<_> = processor_names + .iter() + .map(|processor_name| { + let processor = processors.get(processor_name).cloned(); + async move { + if let Some(processor) = processor { + if processor.status == Status::Active { + if let Some(head_wmb) = processor.timelines[0].get_head_wmb() { + return head_wmb.watermark == 200 && head_wmb.offset == 1; + } + } + } + false + } + }) + .collect(); + + let results = futures::future::join_all(futures).await; + let all_processors_tracked = results.into_iter().all(|tracked| tracked); + + if all_processors_tracked { + break; + } + + if start_time.elapsed() > Duration::from_secs(1) { + panic!("Test failed: Processors were not added or WMBs were not tracked within 1 second"); + } + } + // Abort the tasks + for hb_task in hb_tasks { + hb_task.abort(); + } + for ot_task in ot_tasks { + ot_task.abort(); + } + + // delete the kv store + js_context + .delete_key_value("test_processor_manager_multi_ot_bucket") + .await + .unwrap(); + js_context + .delete_key_value("test_processor_manager_multi_hb_bucket") + .await + .unwrap(); + } +} diff --git a/rust/numaflow-core/src/watermark/processor/timeline.rs b/rust/numaflow-core/src/watermark/processor/timeline.rs new file mode 100644 index 0000000000..ff31738372 --- /dev/null +++ b/rust/numaflow-core/src/watermark/processor/timeline.rs @@ -0,0 +1,358 @@ +//! OffsetTimeline is to store the watermark and offset records. It will always be sorted by watermark +//! from highest to lowest. The timeline will be used to determine the event time for the input offset. +//! Each processor will use this timeline to store the watermark and offset records per input partition. + +use std::cmp::Ordering; +use std::collections::VecDeque; +use std::fmt; +use std::sync::Arc; + +use std::sync::RwLock; +use tracing::{debug, error, info}; + +use crate::watermark::wmb::WMB; + +/// OffsetTimeline is to store the watermark to the offset records. +/// Our list is sorted by event time from highest to lowest. +#[derive(Clone)] +pub(crate) struct OffsetTimeline { + watermarks: Arc>>, + capacity: usize, +} + +impl OffsetTimeline { + pub(crate) fn new(capacity: usize) -> Self { + let mut watermarks = VecDeque::with_capacity(capacity); + for _ in 0..capacity { + watermarks.push_back(WMB::default()); + } + + OffsetTimeline { + watermarks: Arc::new(RwLock::new(watermarks)), + capacity, + } + } + + /// Put inserts the WMB into list. It ensures that the list will remain sorted after the insert. + pub(crate) fn put(&self, node: WMB) { + let mut watermarks = self.watermarks.write().expect("failed to acquire lock"); + + let element_node = watermarks + .front_mut() + .expect("timeline should never be empty"); + + // Different cases: + // 1. Watermark is the same but the offset is larger - valid case, since data is moving forward we should store the larger offset + // 2. Watermark is the same but the offset is smaller - valid case, because of race conditions in the previous processor, we can ignore + // 3. Watermark is larger and the offset is larger - valid case, data is moving forward we should store the larger offset and watermark + // 4. Watermark is larger but the offset is smaller - invalid case, watermark is monotonically increasing for offset + // 5. Watermark is smaller - invalid case, watermark is monotonically increasing per partition and per processor + // 6. Watermark is greater but the offset is the same - valid case, we use same ctrl message to update the watermark, store the new watermark + match ( + node.watermark.cmp(&element_node.watermark), + node.offset.cmp(&element_node.offset), + ) { + (Ordering::Equal, Ordering::Greater) => { + element_node.offset = node.offset; + } + (Ordering::Equal, _) => { + debug!("Watermark the same but input offset smaller than the existing offset - skipping"); + } + (Ordering::Greater, Ordering::Greater) => { + watermarks.push_front(node); + } + (Ordering::Greater, Ordering::Less) => { + error!("The new input offset should never be smaller than the existing offset"); + } + (Ordering::Less, _) => { + error!("Watermark should not regress"); + } + (Ordering::Greater, Ordering::Equal) => { + info!(?node, "Idle Watermark detected"); + element_node.watermark = node.watermark; + } + } + + // trim the timeline + if watermarks.len() > self.capacity { + watermarks.pop_back(); + } + } + + /// GetHeadOffset returns the offset of the head WMB. + pub(crate) fn get_head_offset(&self) -> i64 { + let watermarks = self.watermarks.read().expect("failed to acquire lock"); + watermarks.front().map_or(-1, |w| w.offset) + } + + /// GetHeadWatermark returns the watermark of the head WMB. + pub(crate) fn get_head_watermark(&self) -> i64 { + let watermarks = self.watermarks.read().expect("failed to acquire lock"); + watermarks.front().map_or(-1, |w| w.watermark) + } + + /// GetHeadWMB returns the head WMB. + pub(crate) fn get_head_wmb(&self) -> Option { + let watermarks = self.watermarks.read().expect("failed to acquire lock"); + watermarks.front().copied() + } + + /// GetEventTime returns the event time of the nearest WMB that has an offset less than the input offset. + pub(crate) fn get_event_time(&self, input_offset: i64) -> i64 { + let watermarks = self.watermarks.read().expect("failed to acquire lock"); + watermarks + .iter() + .find(|w| w.offset < input_offset) + .map_or(-1, |w| w.watermark) + } +} + +impl fmt::Debug for OffsetTimeline { + fn fmt(&self, f: &mut fmt::Formatter<'_>) -> fmt::Result { + write!( + f, + "OffsetTimeline {{ capacity: {}, watermarks: {:?} }}", + self.capacity, self.watermarks + ) + } +} + +#[cfg(test)] +mod tests { + use super::*; + use crate::watermark::wmb::WMB; + + #[tokio::test] + async fn test_put_offsets_in_order() { + let timeline = OffsetTimeline::new(10); + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 2, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 250, + offset: 3, + idle: false, + partition: 0, + }; + let wmb4 = WMB { + watermark: 250, + offset: 4, + idle: false, + partition: 0, + }; + let wmb5 = WMB { + watermark: 300, + offset: 5, + idle: false, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); + timeline.put(wmb4); + timeline.put(wmb5); + + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 5); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 300); + + let head_wmb = timeline.get_head_wmb(); + assert!(head_wmb.is_some()); + assert_eq!(head_wmb.expect("failed to acquire lock").watermark, 300); + + let event_time = timeline.get_event_time(3); + assert_eq!(event_time, 200); + } + + #[tokio::test] + async fn test_put_out_of_order_offsets() { + let timeline = OffsetTimeline::new(10); + let wmb1 = WMB { + watermark: 50, + offset: 62, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 100, + offset: 65, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 200, + offset: 63, // out of order should not be considered + idle: false, + partition: 0, + }; + let wmb4 = WMB { + watermark: 250, + offset: 70, + idle: false, + partition: 0, + }; + let wmb5 = WMB { + watermark: 250, + offset: 80, + idle: false, + partition: 0, + }; + let wmb6 = WMB { + watermark: 300, + offset: 86, + idle: false, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); + timeline.put(wmb4); + timeline.put(wmb5); + timeline.put(wmb6); + + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 86); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 300); + + let head_wmb = timeline.get_head_wmb(); + assert!(head_wmb.is_some()); + assert_eq!(head_wmb.expect("failed to acquire lock").watermark, 300); + + let event_time = timeline.get_event_time(65); + assert_eq!(event_time, 50); + } + + #[tokio::test] + async fn test_put_same_watermark_different_offset() { + let timeline = OffsetTimeline::new(10); + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 100, + offset: 2, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 100, + offset: 3, + idle: false, + partition: 0, + }; + let wmb4 = WMB { + watermark: 100, + offset: 4, + idle: false, + partition: 0, + }; + let wmb5 = WMB { + watermark: 100, + offset: 5, + idle: false, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); + timeline.put(wmb4); + timeline.put(wmb5); + + // should only consider the largest offset + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 5); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 100); + + let head_wmb = timeline.get_head_wmb(); + assert!(head_wmb.is_some()); + assert_eq!(head_wmb.expect("failed to acquire lock").watermark, 100); + + // only one entry, so should return -1 + let event_time = timeline.get_event_time(5); + assert_eq!(event_time, -1); + } + + #[tokio::test] + async fn test_put_idle_cases() { + let timeline = OffsetTimeline::new(10); + let wmb1 = WMB { + watermark: 100, + offset: 1, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 2, + idle: false, + partition: 1, + }; + let wmb3 = WMB { + watermark: 150, + offset: 3, + idle: true, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); // should be ignored since the watermark is smaller than the head + + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 2); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 200); + + // valid idle watermark + let idle_wmb = WMB { + watermark: 250, + offset: 4, + idle: true, + partition: 0, + }; + + timeline.put(idle_wmb); + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 4); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 250); + + // same watermark but different offset (larger should be stored) + let idle_wmb = WMB { + watermark: 250, + offset: 5, + idle: true, + partition: 0, + }; + + timeline.put(idle_wmb); + let head_offset = timeline.get_head_offset(); + assert_eq!(head_offset, 5); + + let head_watermark = timeline.get_head_watermark(); + assert_eq!(head_watermark, 250); + } +} diff --git a/rust/numaflow-core/src/watermark/source.rs b/rust/numaflow-core/src/watermark/source.rs new file mode 100644 index 0000000000..2bd1064400 --- /dev/null +++ b/rust/numaflow-core/src/watermark/source.rs @@ -0,0 +1,456 @@ +//! Exposes methods to fetch and publish the watermark for the messages read from [Source], also +//! exposes methods to publish the watermark for the messages written to [ISB]. Since the watermark +//! starts at source we publish and fetch inside the source to get the minimum event time across the +//! partitions. Since we write the messages to ISB, we publish the watermark for the messages written +//! to ISB. Hence, source publisher internally uses ISB publisher to publish the watermarks. Since source +//! is not streaming in nature we don't have to track the inflight offsets and their watermarks. +//! +//! +//! ##### Watermark flow +//! +//! ```text +//! (Read) ---> (Publish WM For Source) ---> (Fetch WM For Source) ---> (Write to ISB) ---> (Publish WM to ISB) +//! ``` +//! +//! [Source]: https://numaflow.numaproj.io/user-guide/sources/overview/ +//! [ISB]: https://numaflow.numaproj.io/core-concepts/inter-step-buffer/ + +use std::collections::HashMap; + +use tokio::sync::mpsc::Receiver; +use tracing::error; + +use crate::config::pipeline::isb::Stream; +use crate::config::pipeline::watermark::SourceWatermarkConfig; +use crate::error::{Error, Result}; +use crate::message::{IntOffset, Message, Offset}; +use crate::watermark::processor::manager::ProcessorManager; +use crate::watermark::source::source_wm_fetcher::SourceWatermarkFetcher; +use crate::watermark::source::source_wm_publisher::SourceWatermarkPublisher; + +/// fetcher for fetching the source watermark +pub(crate) mod source_wm_fetcher; + +/// publisher for publishing the source watermark +pub(crate) mod source_wm_publisher; + +/// Messages that can be sent to the SourceWatermarkActor +enum SourceActorMessage { + PublishSourceWatermark { + map: HashMap, + }, + PublishISBWatermark { + offset: IntOffset, + stream: Stream, + input_partition: u16, + }, +} + +/// SourceWatermarkActor comprises SourcePublisher and SourceFetcher. +struct SourceWatermarkActor { + publisher: SourceWatermarkPublisher, + fetcher: SourceWatermarkFetcher, +} + +impl SourceWatermarkActor { + /// Creates a new SourceWatermarkActor. + fn new(publisher: SourceWatermarkPublisher, fetcher: SourceWatermarkFetcher) -> Self { + Self { publisher, fetcher } + } + + /// Runs the SourceWatermarkActor + async fn run(mut self, mut receiver: Receiver) { + while let Some(message) = receiver.recv().await { + if let Err(e) = self.handle_message(message).await { + error!("error handling message: {:?}", e); + } + } + } + + /// Handles the SourceActorMessage. + async fn handle_message(&mut self, message: SourceActorMessage) -> Result<()> { + match message { + SourceActorMessage::PublishSourceWatermark { map } => { + for (partition, event_time) in map { + self.publisher + .publish_source_watermark(partition, event_time) + .await; + } + } + SourceActorMessage::PublishISBWatermark { + offset, + stream, + input_partition, + } => { + let watermark = self.fetcher.fetch_source_watermark()?; + self.publisher + .publish_isb_watermark( + input_partition, + stream, + offset.offset, + watermark.timestamp_millis(), + ) + .await; + } + } + + Ok(()) + } +} + +/// SourceWatermarkHandle is the handle for the SourceWatermarkActor. +/// Exposes methods to publish the source watermark and edge watermark. +#[derive(Clone)] +pub(crate) struct SourceWatermarkHandle { + sender: tokio::sync::mpsc::Sender, +} + +impl SourceWatermarkHandle { + /// Creates a new SourceWatermarkHandle. + pub(crate) async fn new( + js_context: async_nats::jetstream::Context, + config: &SourceWatermarkConfig, + ) -> Result { + let (sender, receiver) = tokio::sync::mpsc::channel(100); + let processor_manager = + ProcessorManager::new(js_context.clone(), &config.source_bucket_config).await?; + + let fetcher = SourceWatermarkFetcher::new(processor_manager); + let publisher = SourceWatermarkPublisher::new( + js_context.clone(), + config.source_bucket_config.clone(), + config.to_vertex_bucket_config.clone(), + ) + .await + .map_err(|e| Error::Watermark(e.to_string()))?; + + let actor = SourceWatermarkActor::new(publisher, fetcher); + tokio::spawn(async move { actor.run(receiver).await }); + Ok(Self { sender }) + } + + /// Generates and Publishes the source watermark for the given messages. + pub(crate) async fn generate_and_publish_source_watermark( + &self, + messages: &[Message], + ) -> Result<()> { + // we need to build a hash-map of the lowest event time for each partition + let partition_to_lowest_event_time = + messages.iter().fold(HashMap::new(), |mut acc, message| { + let partition_id = match &message.offset { + Offset::Int(offset) => offset.partition_idx, + Offset::String(offset) => offset.partition_idx, + }; + + let event_time = message.event_time.timestamp_millis(); + + let lowest_event_time = acc.entry(partition_id).or_insert(event_time); + if event_time < *lowest_event_time { + *lowest_event_time = event_time; + } + acc + }); + + self.sender + .send(SourceActorMessage::PublishSourceWatermark { + map: partition_to_lowest_event_time, + }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + + Ok(()) + } + + /// Publishes the watermark for the given input partition on to the ISB of the next vertex. + pub(crate) async fn publish_source_isb_watermark( + &self, + stream: Stream, + offset: Offset, + input_partition: u16, + ) -> Result<()> { + // the fetching happens in the handler + if let Offset::Int(offset) = offset { + self.sender + .send(SourceActorMessage::PublishISBWatermark { + offset, + stream, + input_partition, + }) + .await + .map_err(|_| Error::Watermark("failed to send message".to_string()))?; + + Ok(()) + } else { + Err(Error::Watermark("invalid offset type".to_string())) + } + } +} + +#[cfg(test)] +mod tests { + use async_nats::jetstream; + use async_nats::jetstream::kv::Config; + use chrono::DateTime; + + use super::*; + use crate::config::pipeline::isb::Stream; + use crate::config::pipeline::watermark::BucketConfig; + use crate::message::{IntOffset, Message}; + use crate::watermark::wmb::WMB; + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_source_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "test_publish_source_watermark_OT"; + let hb_bucket_name = "test_publish_source_watermark_PROCESSORS"; + + let source_config = SourceWatermarkConfig { + source_bucket_config: BucketConfig { + vertex: "source_vertex", + partitions: 1, // partitions is always one for source + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }, + to_vertex_bucket_config: vec![], + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let handle = SourceWatermarkHandle::new(js_context.clone(), &source_config) + .await + .expect("Failed to create source watermark handle"); + + let messages = vec![ + Message { + offset: Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + }), + event_time: DateTime::from_timestamp_millis(60000).unwrap(), + ..Default::default() + }, + Message { + offset: Offset::Int(IntOffset { + offset: 2, + partition_idx: 0, + }), + event_time: DateTime::from_timestamp_millis(70000).unwrap(), + ..Default::default() + }, + ]; + + handle + .generate_and_publish_source_watermark(&messages) + .await + .expect("Failed to publish source watermark"); + + // try getting the value for the processor from the ot bucket to make sure + // the watermark is getting published(min event time in the batch), wait until + // one second if it's not there, fail the test + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let mut wmb_found = false; + for _ in 0..10 { + let wmb = ot_bucket + .get("source_vertex-0") + .await + .expect("Failed to get wmb"); + if wmb.is_some() { + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.watermark, 60000); + wmb_found = true; + break; + } else { + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + } + } + + if !wmb_found { + panic!("Failed to get watermark"); + } + + // delete the stores + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_source_edge_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let source_ot_bucket_name = "test_publish_source_edge_watermark_source_OT"; + let source_hb_bucket_name = "test_publish_source_edge_watermark_source_PROCESSORS"; + let edge_ot_bucket_name = "test_publish_source_edge_watermark_edge_OT"; + let edge_hb_bucket_name = "test_publish_source_edge_watermark_edge_PROCESSORS"; + + let source_config = SourceWatermarkConfig { + source_bucket_config: BucketConfig { + vertex: "source_vertex", + partitions: 2, + ot_bucket: source_ot_bucket_name, + hb_bucket: source_hb_bucket_name, + }, + to_vertex_bucket_config: vec![BucketConfig { + vertex: "edge_vertex", + partitions: 2, + ot_bucket: edge_ot_bucket_name, + hb_bucket: edge_hb_bucket_name, + }], + }; + + // create key value stores for source + js_context + .create_key_value(Config { + bucket: source_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: source_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + // create key value stores for edge + js_context + .create_key_value(Config { + bucket: edge_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: edge_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let handle = SourceWatermarkHandle::new(js_context.clone(), &source_config) + .await + .expect("Failed to create source watermark handle"); + + let ot_bucket = js_context + .get_key_value(edge_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let stream = Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }; + + let mut wmb_found = false; + for i in 1..11 { + // publish source watermarks before publishing edge watermarks + let messages = vec![ + Message { + offset: Offset::Int(IntOffset { + offset: 1, + partition_idx: 0, + }), + event_time: DateTime::from_timestamp_millis(10000 * i).unwrap(), + ..Default::default() + }, + Message { + offset: Offset::Int(IntOffset { + offset: 2, + partition_idx: 0, + }), + event_time: DateTime::from_timestamp_millis(20000 * i).unwrap(), + ..Default::default() + }, + ]; + + handle + .generate_and_publish_source_watermark(&messages) + .await + .expect("Failed to publish source watermark"); + + let offset = Offset::Int(IntOffset { + offset: i, + partition_idx: 0, + }); + handle + .publish_source_isb_watermark(stream.clone(), offset, 0) + .await + .expect("Failed to publish edge watermark"); + + // check if the watermark is published + let wmb = ot_bucket + .get("source_vertex-edge_vertex-0") + .await + .expect("Failed to get wmb"); + if wmb.is_some() { + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_ne!(wmb.watermark, -1); + wmb_found = true; + break; + } else { + tokio::time::sleep(tokio::time::Duration::from_millis(100)).await; + } + } + + if !wmb_found { + panic!("Failed to get watermark"); + } + + // delete the stores + js_context + .delete_key_value(source_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(source_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_ot_bucket_name.to_string()) + .await + .unwrap(); + } +} diff --git a/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs b/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs new file mode 100644 index 0000000000..67ad7716bc --- /dev/null +++ b/rust/numaflow-core/src/watermark/source/source_wm_fetcher.rs @@ -0,0 +1,193 @@ +//! Fetches watermark for the source, source fetcher will only have one processor manager which tracks +//! all the partition based processors. The fetcher will look at all the active processors and their +//! timelines to determine the watermark. We don't care about offsets here since the watermark starts +//! at source, we only consider the head watermark and consider the minimum watermark of all the active +//! processors. +use crate::error::Result; +use crate::watermark::processor::manager::ProcessorManager; +use crate::watermark::wmb::Watermark; + +/// SourceWatermarkFetcher is the watermark fetcher for the source. +pub struct SourceWatermarkFetcher { + processor_manager: ProcessorManager, +} + +impl SourceWatermarkFetcher { + /// Creates a new [SourceWatermarkFetcher]. + pub(crate) fn new(processor_manager: ProcessorManager) -> Self { + SourceWatermarkFetcher { processor_manager } + } + + /// Fetches the watermark for the source, which is the minimum watermark of all the active + /// processors. + pub(crate) fn fetch_source_watermark(&mut self) -> Result { + let mut min_wm = i64::MAX; + + for (_, processor) in self + .processor_manager + .processors + .read() + .expect("failed to acquire lock") + .iter() + { + // We only consider active processors. + if !processor.is_active() { + continue; + } + + // only consider the head watermark of the processor + let head_wm = processor.timelines.first().unwrap().get_head_watermark(); + + if head_wm < min_wm { + min_wm = head_wm; + } + } + + if min_wm == i64::MAX { + min_wm = -1; + } + + Ok(Watermark::from_timestamp_millis(min_wm).expect("Invalid watermark")) + } +} + +#[cfg(test)] +mod tests { + use std::collections::HashMap; + use std::sync::Arc; + + use bytes::Bytes; + use std::sync::RwLock; + + use super::*; + use crate::watermark::processor::manager::{Processor, Status}; + use crate::watermark::processor::timeline::OffsetTimeline; + use crate::watermark::wmb::WMB; + + #[tokio::test] + async fn test_source_watermark_fetcher_single_processor() { + // Create a ProcessorManager with a single Processor and a single OffsetTimeline + let processor_name = Bytes::from("processor1"); + let mut processor = Processor::new(processor_name.clone(), Status::Active, 1); + let timeline = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb1 = WMB { + watermark: 100, + offset: 19723492734, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 19723492735, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 300, + offset: 19723492736, + idle: false, + partition: 0, + }; + + timeline.put(wmb1); + timeline.put(wmb2); + timeline.put(wmb3); + + processor.timelines[0] = timeline; + + let mut processors = HashMap::new(); + processors.insert(processor_name.clone(), processor); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut fetcher = SourceWatermarkFetcher::new(processor_manager); + + // Invoke fetch_watermark and verify the result + let watermark = fetcher.fetch_source_watermark().unwrap(); + assert_eq!(watermark.timestamp_millis(), 300); + } + + #[tokio::test] + async fn test_source_watermark_fetcher_multi_processors() { + // Create a ProcessorManager with multiple Processors and a single OffsetTimeline + let processor_name1 = Bytes::from("processor1"); + let mut processor1 = Processor::new(processor_name1.clone(), Status::Active, 1); + let timeline1 = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb1 = WMB { + watermark: 100, + offset: 19723492734, + idle: false, + partition: 0, + }; + let wmb2 = WMB { + watermark: 200, + offset: 19723492735, + idle: false, + partition: 0, + }; + let wmb3 = WMB { + watermark: 323, + offset: 19723492736, + idle: false, + partition: 0, + }; + + timeline1.put(wmb1); + timeline1.put(wmb2); + timeline1.put(wmb3); + + processor1.timelines[0] = timeline1; + + let processor_name2 = Bytes::from("processor2"); + let mut processor2 = Processor::new(processor_name2.clone(), Status::Active, 1); + let timeline2 = OffsetTimeline::new(10); + + // Populate the OffsetTimeline with sorted WMB entries + let wmb4 = WMB { + watermark: 150, + offset: 19723492734, + idle: false, + partition: 0, + }; + let wmb5 = WMB { + watermark: 250, + offset: 19723492735, + idle: false, + partition: 0, + }; + let wmb6 = WMB { + watermark: 350, + offset: 19723492736, + idle: false, + partition: 0, + }; + + timeline2.put(wmb4); + timeline2.put(wmb5); + timeline2.put(wmb6); + + processor2.timelines[0] = timeline2; + + let mut processors = HashMap::new(); + processors.insert(processor_name1.clone(), processor1); + processors.insert(processor_name2.clone(), processor2); + + let processor_manager = ProcessorManager { + processors: Arc::new(RwLock::new(processors)), + handles: vec![], + }; + + let mut fetcher = SourceWatermarkFetcher::new(processor_manager); + + // Invoke fetch_watermark and verify the result + let watermark = fetcher.fetch_source_watermark().unwrap(); + assert_eq!(watermark.timestamp_millis(), 323); + } +} diff --git a/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs b/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs new file mode 100644 index 0000000000..f20d2e3fb9 --- /dev/null +++ b/rust/numaflow-core/src/watermark/source/source_wm_publisher.rs @@ -0,0 +1,304 @@ +//! Publishes the watermark for source, since watermark originates at source, we publish and fetch to determine +//! the watermark across the source partitions. Since we write the messages to the ISB, we will also publish +//! the watermark to the ISB. Unlike other vertices we don't use pod as the processing entity for publishing +//! watermark we use the partition(watermark originates here). +use std::collections::HashMap; + +use chrono::Utc; +use tracing::info; + +use crate::config::pipeline::isb::Stream; +use crate::config::pipeline::watermark::BucketConfig; +use crate::error; +use crate::watermark::isb::wm_publisher::ISBWatermarkPublisher; + +/// SourcePublisher is the watermark publisher for the source vertex. +pub(crate) struct SourceWatermarkPublisher { + js_context: async_nats::jetstream::Context, + source_config: BucketConfig, + to_vertex_configs: Vec, + publishers: HashMap, +} + +impl SourceWatermarkPublisher { + /// Creates a new [SourceWatermarkPublisher]. + pub(crate) async fn new( + js_context: async_nats::jetstream::Context, + source_config: BucketConfig, + to_vertex_configs: Vec, + ) -> error::Result { + Ok(SourceWatermarkPublisher { + js_context, + source_config, + to_vertex_configs, + publishers: HashMap::new(), + }) + } + + /// Publishes the source watermark for the input partition. It internally uses edge publisher + /// with processor set to the input partition and source OT. + pub(crate) async fn publish_source_watermark(&mut self, partition: u16, watermark: i64) { + // for source, we do partition-based watermark publishing rather than pod-based, hence + // the processing entity is the partition itself. We create a publisher for each partition + // and publish the watermark to it. + let processor_name = format!("{}-{}", self.source_config.vertex, partition); + // create a publisher if not exists + if !self.publishers.contains_key(&processor_name) { + let publisher = ISBWatermarkPublisher::new( + processor_name.clone(), + self.js_context.clone(), + &[self.source_config.clone()], + ) + .await + .expect("Failed to create publisher"); + info!(processor = ?processor_name, partittion = ?partition, + "Creating new publisher for source" + ); + self.publishers.insert(processor_name.clone(), publisher); + } + + self.publishers + .get_mut(&processor_name) + .expect("Publisher not found") + .publish_watermark( + Stream { + name: "source", + vertex: self.source_config.vertex, + partition, + }, + Utc::now().timestamp_micros(), // we don't care about the offsets + watermark, + ) + .await + .expect("Failed to publish watermark"); + } + + /// Publishes the ISB watermark for the input partition. It internally uses ISB publisher with + /// processor set to the input partition and ISB OTs. + pub(crate) async fn publish_isb_watermark( + &mut self, + input_partition: u16, + stream: Stream, + offset: i64, + watermark: i64, + ) { + let processor_name = format!( + "{}-{}-{}", + self.source_config.vertex, stream.vertex, input_partition + ); + // In source, since we do partition-based watermark publishing rather than pod-based, we + // create a publisher for each partition and publish the watermark to it. + if !self.publishers.contains_key(&processor_name) { + info!(processor = ?processor_name, partition = ?input_partition, + "Creating new publisher for ISB" + ); + let publisher = ISBWatermarkPublisher::new( + processor_name.clone(), + self.js_context.clone(), + &self.to_vertex_configs, + ) + .await + .expect("Failed to create publisher"); + self.publishers.insert(processor_name.clone(), publisher); + } + + self.publishers + .get_mut(&processor_name) + .expect("Publisher not found") + .publish_watermark(stream, offset, watermark) + .await + .expect("Failed to publish watermark"); + } +} + +#[cfg(test)] +mod tests { + use async_nats::jetstream; + use async_nats::jetstream::kv::Config; + + use crate::config::pipeline::isb::Stream; + use crate::watermark::source::source_wm_publisher::{BucketConfig, SourceWatermarkPublisher}; + use crate::watermark::wmb::WMB; + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_source_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let ot_bucket_name = "source_watermark_OT"; + let hb_bucket_name = "source_watermark_PROCESSORS"; + + let source_config = BucketConfig { + vertex: "source_vertex", + partitions: 2, + ot_bucket: ot_bucket_name, + hb_bucket: hb_bucket_name, + }; + + // create key value stores + js_context + .create_key_value(Config { + bucket: ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + js_context + .create_key_value(Config { + bucket: hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut source_publisher = + SourceWatermarkPublisher::new(js_context.clone(), source_config.clone(), vec![]) + .await + .expect("Failed to create source publisher"); + + // Publish source watermark for partition 0 + source_publisher.publish_source_watermark(0, 100).await; + + let ot_bucket = js_context + .get_key_value(ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("source_vertex-0") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.watermark, 100); + + // delete the stores + js_context + .delete_key_value(hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(ot_bucket_name.to_string()) + .await + .unwrap(); + } + + #[cfg(feature = "nats-tests")] + #[tokio::test] + async fn test_publish_edge_watermark() { + let client = async_nats::connect("localhost:4222").await.unwrap(); + let js_context = jetstream::new(client); + + let source_ot_bucket_name = "source_edge_watermark_source_OT"; + let source_hb_bucket_name = "source_edge_watermark_source_PROCESSORS"; + let edge_ot_bucket_name = "source_edge_watermark_edge_OT"; + let edge_hb_bucket_name = "source_edge_watermark_edge_PROCESSORS"; + + let source_config = BucketConfig { + vertex: "source_vertex", + partitions: 2, + ot_bucket: source_ot_bucket_name, + hb_bucket: source_hb_bucket_name, + }; + + let edge_config = BucketConfig { + vertex: "edge_vertex", + partitions: 2, + ot_bucket: edge_ot_bucket_name, + hb_bucket: edge_hb_bucket_name, + }; + + // create key value stores for source + js_context + .create_key_value(Config { + bucket: source_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: source_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + // create key value stores for edge + js_context + .create_key_value(Config { + bucket: edge_ot_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + js_context + .create_key_value(Config { + bucket: edge_hb_bucket_name.to_string(), + history: 1, + ..Default::default() + }) + .await + .unwrap(); + + let mut source_publisher = SourceWatermarkPublisher::new( + js_context.clone(), + source_config.clone(), + vec![edge_config.clone()], + ) + .await + .expect("Failed to create source publisher"); + + let stream = Stream { + name: "edge_stream", + vertex: "edge_vertex", + partition: 0, + }; + + // Publish edge watermark for partition 0 + source_publisher + .publish_isb_watermark(0, stream.clone(), 1, 200) + .await; + + let ot_bucket = js_context + .get_key_value(edge_ot_bucket_name) + .await + .expect("Failed to get ot bucket"); + + let wmb = ot_bucket + .get("source_vertex-edge_vertex-0") + .await + .expect("Failed to get wmb"); + assert!(wmb.is_some()); + + let wmb: WMB = wmb.unwrap().try_into().unwrap(); + assert_eq!(wmb.offset, 1); + assert_eq!(wmb.watermark, 200); + + // delete the stores + js_context + .delete_key_value(source_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(source_ot_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_hb_bucket_name.to_string()) + .await + .unwrap(); + js_context + .delete_key_value(edge_ot_bucket_name.to_string()) + .await + .unwrap(); + } +} diff --git a/rust/numaflow-core/src/watermark/wmb.rs b/rust/numaflow-core/src/watermark/wmb.rs new file mode 100644 index 0000000000..fbc1652be7 --- /dev/null +++ b/rust/numaflow-core/src/watermark/wmb.rs @@ -0,0 +1,67 @@ +use bytes::{Bytes, BytesMut}; +use chrono::{DateTime, Utc}; +use prost::Message; + +use crate::error::Error; + +/// WMB is the watermark message that is sent by the processor to the downstream. +#[derive(Clone, Copy, Debug, PartialEq)] +#[allow(clippy::upper_case_acronyms)] +pub(crate) struct WMB { + pub(crate) idle: bool, + pub(crate) offset: i64, + pub(crate) watermark: i64, + pub(crate) partition: u16, +} + +impl Default for WMB { + fn default() -> Self { + Self { + watermark: -1, + offset: -1, + idle: false, + partition: 0, + } + } +} + +/// Watermark is a monotonically increasing time. +pub(crate) type Watermark = DateTime; + +/// Converts a protobuf bytes to WMB. +impl TryFrom for WMB { + type Error = Error; + + fn try_from(bytes: Bytes) -> Result { + let proto_wmb = numaflow_pb::objects::watermark::Wmb::decode(bytes) + .map_err(|e| Error::Proto(e.to_string()))?; + + Ok(WMB { + idle: proto_wmb.idle, + offset: proto_wmb.offset, + watermark: proto_wmb.watermark, + partition: proto_wmb.partition as u16, + }) + } +} + +/// Converts WMB to protobuf bytes. +impl TryFrom for BytesMut { + type Error = Error; + + fn try_from(wmb: WMB) -> Result { + let mut bytes = BytesMut::new(); + let proto_wmb = numaflow_pb::objects::watermark::Wmb { + idle: wmb.idle, + offset: wmb.offset, + watermark: wmb.watermark, + partition: wmb.partition as i32, + }; + + proto_wmb + .encode(&mut bytes) + .map_err(|e| Error::Proto(e.to_string()))?; + + Ok(bytes) + } +} diff --git a/rust/numaflow-pb/src/main.rs b/rust/numaflow-pb/src/main.rs index 80e2390893..c8af01b5a1 100644 --- a/rust/numaflow-pb/src/main.rs +++ b/rust/numaflow-pb/src/main.rs @@ -31,7 +31,7 @@ fn build_objects() { prost_build::Config::new() .out_dir("src/objects") .compile_protos( - &["proto/isb/message.proto", "proto/wmb/wmb.proto"], + &["proto/isb/message.proto", "proto/watermark/watermark.proto"], &["proto"], ) .expect("failed to compile protos"); diff --git a/rust/numaflow-pb/src/objects.rs b/rust/numaflow-pb/src/objects.rs index f30f255ca1..9f5edf7e2c 100644 --- a/rust/numaflow-pb/src/objects.rs +++ b/rust/numaflow-pb/src/objects.rs @@ -2,6 +2,6 @@ #[rustfmt::skip] pub mod isb; -#[path = "objects/wmb.rs"] +#[path = "objects/watermark.rs"] #[rustfmt::skip] -pub mod wmb; +pub mod watermark; diff --git a/rust/numaflow-pb/src/objects/wmb.rs b/rust/numaflow-pb/src/objects/watermark.rs similarity index 81% rename from rust/numaflow-pb/src/objects/wmb.rs rename to rust/numaflow-pb/src/objects/watermark.rs index f60b4a0d07..2b5e837b1e 100644 --- a/rust/numaflow-pb/src/objects/wmb.rs +++ b/rust/numaflow-pb/src/objects/watermark.rs @@ -21,3 +21,10 @@ pub struct Wmb { #[prost(int32, tag = "4")] pub partition: i32, } +/// Heartbeat is used to track the active processors +#[derive(Clone, Copy, PartialEq, ::prost::Message)] +pub struct Heartbeat { + /// Heartbeat(current time in millis) published by the active processors. + #[prost(int64, tag = "1")] + pub heartbeat: i64, +} diff --git a/rust/numaflow/src/main.rs b/rust/numaflow/src/main.rs index 56f40ca48c..d2ef238bce 100644 --- a/rust/numaflow/src/main.rs +++ b/rust/numaflow/src/main.rs @@ -2,12 +2,13 @@ use std::env; use std::error::Error; use std::time::Duration; -use tracing::error; +use tracing::{error, info}; use tracing_subscriber::layer::SubscriberExt; use tracing_subscriber::util::SubscriberInitExt; #[tokio::main] async fn main() -> Result<(), Box> { + info!("Starting numaflow"); // Set up the tracing subscriber. RUST_LOG can be used to set the log level. // The default log level is `info`. The `axum::rejection=trace` enables showing // rejections from built-in extractors at `TRACE` level. diff --git a/rust/serving/src/app/callback/state.rs b/rust/serving/src/app/callback/state.rs index 2354899aa9..a1c1869da8 100644 --- a/rust/serving/src/app/callback/state.rs +++ b/rust/serving/src/app/callback/state.rs @@ -226,11 +226,12 @@ where #[cfg(test)] mod tests { + use axum::body::Bytes; + use super::*; use crate::app::callback::store::memstore::InMemoryStore; use crate::callback::Response; use crate::pipeline::PipelineDCG; - use axum::body::Bytes; const PIPELINE_SPEC_ENCODED: &str = "eyJ2ZXJ0aWNlcyI6W3sibmFtZSI6ImluIiwic291cmNlIjp7InNlcnZpbmciOnsiYXV0aCI6bnVsbCwic2VydmljZSI6dHJ1ZSwibXNnSURIZWFkZXJLZXkiOiJYLU51bWFmbG93LUlkIiwic3RvcmUiOnsidXJsIjoicmVkaXM6Ly9yZWRpczo2Mzc5In19fSwiY29udGFpbmVyVGVtcGxhdGUiOnsicmVzb3VyY2VzIjp7fSwiaW1hZ2VQdWxsUG9saWN5IjoiTmV2ZXIiLCJlbnYiOlt7Im5hbWUiOiJSVVNUX0xPRyIsInZhbHVlIjoiZGVidWcifV19LCJzY2FsZSI6eyJtaW4iOjF9LCJ1cGRhdGVTdHJhdGVneSI6eyJ0eXBlIjoiUm9sbGluZ1VwZGF0ZSIsInJvbGxpbmdVcGRhdGUiOnsibWF4VW5hdmFpbGFibGUiOiIyNSUifX19LHsibmFtZSI6InBsYW5uZXIiLCJ1ZGYiOnsiY29udGFpbmVyIjp7ImltYWdlIjoiYXNjaWk6MC4xIiwiYXJncyI6WyJwbGFubmVyIl0sInJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn0sImJ1aWx0aW4iOm51bGwsImdyb3VwQnkiOm51bGx9LCJjb250YWluZXJUZW1wbGF0ZSI6eyJyZXNvdXJjZXMiOnt9LCJpbWFnZVB1bGxQb2xpY3kiOiJOZXZlciJ9LCJzY2FsZSI6eyJtaW4iOjF9LCJ1cGRhdGVTdHJhdGVneSI6eyJ0eXBlIjoiUm9sbGluZ1VwZGF0ZSIsInJvbGxpbmdVcGRhdGUiOnsibWF4VW5hdmFpbGFibGUiOiIyNSUifX19LHsibmFtZSI6InRpZ2VyIiwidWRmIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6ImFzY2lpOjAuMSIsImFyZ3MiOlsidGlnZXIiXSwicmVzb3VyY2VzIjp7fSwiaW1hZ2VQdWxsUG9saWN5IjoiTmV2ZXIifSwiYnVpbHRpbiI6bnVsbCwiZ3JvdXBCeSI6bnVsbH0sImNvbnRhaW5lclRlbXBsYXRlIjp7InJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn0sInNjYWxlIjp7Im1pbiI6MX0sInVwZGF0ZVN0cmF0ZWd5Ijp7InR5cGUiOiJSb2xsaW5nVXBkYXRlIiwicm9sbGluZ1VwZGF0ZSI6eyJtYXhVbmF2YWlsYWJsZSI6IjI1JSJ9fX0seyJuYW1lIjoiZG9nIiwidWRmIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6ImFzY2lpOjAuMSIsImFyZ3MiOlsiZG9nIl0sInJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn0sImJ1aWx0aW4iOm51bGwsImdyb3VwQnkiOm51bGx9LCJjb250YWluZXJUZW1wbGF0ZSI6eyJyZXNvdXJjZXMiOnt9LCJpbWFnZVB1bGxQb2xpY3kiOiJOZXZlciJ9LCJzY2FsZSI6eyJtaW4iOjF9LCJ1cGRhdGVTdHJhdGVneSI6eyJ0eXBlIjoiUm9sbGluZ1VwZGF0ZSIsInJvbGxpbmdVcGRhdGUiOnsibWF4VW5hdmFpbGFibGUiOiIyNSUifX19LHsibmFtZSI6ImVsZXBoYW50IiwidWRmIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6ImFzY2lpOjAuMSIsImFyZ3MiOlsiZWxlcGhhbnQiXSwicmVzb3VyY2VzIjp7fSwiaW1hZ2VQdWxsUG9saWN5IjoiTmV2ZXIifSwiYnVpbHRpbiI6bnVsbCwiZ3JvdXBCeSI6bnVsbH0sImNvbnRhaW5lclRlbXBsYXRlIjp7InJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn0sInNjYWxlIjp7Im1pbiI6MX0sInVwZGF0ZVN0cmF0ZWd5Ijp7InR5cGUiOiJSb2xsaW5nVXBkYXRlIiwicm9sbGluZ1VwZGF0ZSI6eyJtYXhVbmF2YWlsYWJsZSI6IjI1JSJ9fX0seyJuYW1lIjoiYXNjaWlhcnQiLCJ1ZGYiOnsiY29udGFpbmVyIjp7ImltYWdlIjoiYXNjaWk6MC4xIiwiYXJncyI6WyJhc2NpaWFydCJdLCJyZXNvdXJjZXMiOnt9LCJpbWFnZVB1bGxQb2xpY3kiOiJOZXZlciJ9LCJidWlsdGluIjpudWxsLCJncm91cEJ5IjpudWxsfSwiY29udGFpbmVyVGVtcGxhdGUiOnsicmVzb3VyY2VzIjp7fSwiaW1hZ2VQdWxsUG9saWN5IjoiTmV2ZXIifSwic2NhbGUiOnsibWluIjoxfSwidXBkYXRlU3RyYXRlZ3kiOnsidHlwZSI6IlJvbGxpbmdVcGRhdGUiLCJyb2xsaW5nVXBkYXRlIjp7Im1heFVuYXZhaWxhYmxlIjoiMjUlIn19fSx7Im5hbWUiOiJzZXJ2ZS1zaW5rIiwic2luayI6eyJ1ZHNpbmsiOnsiY29udGFpbmVyIjp7ImltYWdlIjoic2VydmVzaW5rOjAuMSIsImVudiI6W3sibmFtZSI6Ik5VTUFGTE9XX0NBTExCQUNLX1VSTF9LRVkiLCJ2YWx1ZSI6IlgtTnVtYWZsb3ctQ2FsbGJhY2stVXJsIn0seyJuYW1lIjoiTlVNQUZMT1dfTVNHX0lEX0hFQURFUl9LRVkiLCJ2YWx1ZSI6IlgtTnVtYWZsb3ctSWQifV0sInJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn19LCJyZXRyeVN0cmF0ZWd5Ijp7fX0sImNvbnRhaW5lclRlbXBsYXRlIjp7InJlc291cmNlcyI6e30sImltYWdlUHVsbFBvbGljeSI6Ik5ldmVyIn0sInNjYWxlIjp7Im1pbiI6MX0sInVwZGF0ZVN0cmF0ZWd5Ijp7InR5cGUiOiJSb2xsaW5nVXBkYXRlIiwicm9sbGluZ1VwZGF0ZSI6eyJtYXhVbmF2YWlsYWJsZSI6IjI1JSJ9fX0seyJuYW1lIjoiZXJyb3Itc2luayIsInNpbmsiOnsidWRzaW5rIjp7ImNvbnRhaW5lciI6eyJpbWFnZSI6InNlcnZlc2luazowLjEiLCJlbnYiOlt7Im5hbWUiOiJOVU1BRkxPV19DQUxMQkFDS19VUkxfS0VZIiwidmFsdWUiOiJYLU51bWFmbG93LUNhbGxiYWNrLVVybCJ9LHsibmFtZSI6Ik5VTUFGTE9XX01TR19JRF9IRUFERVJfS0VZIiwidmFsdWUiOiJYLU51bWFmbG93LUlkIn1dLCJyZXNvdXJjZXMiOnt9LCJpbWFnZVB1bGxQb2xpY3kiOiJOZXZlciJ9fSwicmV0cnlTdHJhdGVneSI6e319LCJjb250YWluZXJUZW1wbGF0ZSI6eyJyZXNvdXJjZXMiOnt9LCJpbWFnZVB1bGxQb2xpY3kiOiJOZXZlciJ9LCJzY2FsZSI6eyJtaW4iOjF9LCJ1cGRhdGVTdHJhdGVneSI6eyJ0eXBlIjoiUm9sbGluZ1VwZGF0ZSIsInJvbGxpbmdVcGRhdGUiOnsibWF4VW5hdmFpbGFibGUiOiIyNSUifX19XSwiZWRnZXMiOlt7ImZyb20iOiJpbiIsInRvIjoicGxhbm5lciIsImNvbmRpdGlvbnMiOm51bGx9LHsiZnJvbSI6InBsYW5uZXIiLCJ0byI6ImFzY2lpYXJ0IiwiY29uZGl0aW9ucyI6eyJ0YWdzIjp7Im9wZXJhdG9yIjoib3IiLCJ2YWx1ZXMiOlsiYXNjaWlhcnQiXX19fSx7ImZyb20iOiJwbGFubmVyIiwidG8iOiJ0aWdlciIsImNvbmRpdGlvbnMiOnsidGFncyI6eyJvcGVyYXRvciI6Im9yIiwidmFsdWVzIjpbInRpZ2VyIl19fX0seyJmcm9tIjoicGxhbm5lciIsInRvIjoiZG9nIiwiY29uZGl0aW9ucyI6eyJ0YWdzIjp7Im9wZXJhdG9yIjoib3IiLCJ2YWx1ZXMiOlsiZG9nIl19fX0seyJmcm9tIjoicGxhbm5lciIsInRvIjoiZWxlcGhhbnQiLCJjb25kaXRpb25zIjp7InRhZ3MiOnsib3BlcmF0b3IiOiJvciIsInZhbHVlcyI6WyJlbGVwaGFudCJdfX19LHsiZnJvbSI6InRpZ2VyIiwidG8iOiJzZXJ2ZS1zaW5rIiwiY29uZGl0aW9ucyI6bnVsbH0seyJmcm9tIjoiZG9nIiwidG8iOiJzZXJ2ZS1zaW5rIiwiY29uZGl0aW9ucyI6bnVsbH0seyJmcm9tIjoiZWxlcGhhbnQiLCJ0byI6InNlcnZlLXNpbmsiLCJjb25kaXRpb25zIjpudWxsfSx7ImZyb20iOiJhc2NpaWFydCIsInRvIjoic2VydmUtc2luayIsImNvbmRpdGlvbnMiOm51bGx9LHsiZnJvbSI6InBsYW5uZXIiLCJ0byI6ImVycm9yLXNpbmsiLCJjb25kaXRpb25zIjp7InRhZ3MiOnsib3BlcmF0b3IiOiJvciIsInZhbHVlcyI6WyJlcnJvciJdfX19XSwibGlmZWN5Y2xlIjp7fSwid2F0ZXJtYXJrIjp7fX0="; diff --git a/rust/serving/src/app/callback/store/redisstore.rs b/rust/serving/src/app/callback/store/redisstore.rs index 490b7be940..2716f0ad13 100644 --- a/rust/serving/src/app/callback/store/redisstore.rs +++ b/rust/serving/src/app/callback/store/redisstore.rs @@ -201,11 +201,12 @@ impl super::Store for RedisConnection { #[cfg(feature = "redis-tests")] #[cfg(test)] mod tests { + use axum::body::Bytes; + use redis::AsyncCommands; + use super::*; use crate::app::callback::store::LocalStore; use crate::callback::Response; - use axum::body::Bytes; - use redis::AsyncCommands; #[tokio::test] async fn test_redis_store() { diff --git a/rust/serving/src/callback.rs b/rust/serving/src/callback.rs index d3f7ba893d..9b64284f14 100644 --- a/rust/serving/src/callback.rs +++ b/rust/serving/src/callback.rs @@ -163,6 +163,16 @@ impl CallbackHandler { #[cfg(test)] mod tests { + use std::sync::atomic::{AtomicUsize, Ordering}; + use std::sync::Arc; + use std::time::Duration; + + use axum::http::StatusCode; + use axum::routing::{get, post}; + use axum::{Json, Router}; + use axum_server::tls_rustls::RustlsConfig; + use tokio::sync::mpsc; + use crate::app::callback::state::State as CallbackState; use crate::app::callback::store::memstore::InMemoryStore; use crate::app::start_main_server; @@ -172,14 +182,6 @@ mod tests { use crate::pipeline::PipelineDCG; use crate::test_utils::get_port; use crate::{AppState, Settings}; - use axum::http::StatusCode; - use axum::routing::{get, post}; - use axum::{Json, Router}; - use axum_server::tls_rustls::RustlsConfig; - use std::sync::atomic::{AtomicUsize, Ordering}; - use std::sync::Arc; - use std::time::Duration; - use tokio::sync::mpsc; type Result = std::result::Result>; @@ -245,7 +247,7 @@ mod tests { // On the server, this fails with SubGraphInvalidInput("Invalid callback: 1234, vertex: in") // We get 200 OK response from the server, since we already registered this request ID in the store. - let callback_task = callback_handler + callback_handler .callback( ID_VALUE.into(), format!("https://localhost:{port}/v1/process/callback"), @@ -253,8 +255,17 @@ mod tests { vec![], ) .await?; - callback_task.await.unwrap(); - let data = store.data.lock().unwrap().get(ID_VALUE).cloned(); + let mut data = None; + for _ in 0..10 { + tokio::time::sleep(Duration::from_millis(2)).await; + data = { + let guard = store.data.lock().unwrap(); + guard.get(ID_VALUE).cloned() + }; + if data.is_some() { + break; + } + } assert!(data.is_some(), "Callback data not found in store"); server_handle.abort(); Ok(())