From 2177d6213184d4a0eebc69981c1d363848d09b42 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Wed, 11 Jan 2023 09:41:26 -0800 Subject: [PATCH 01/13] doc: reduce persistent store (#458) Signed-off-by: Vigith Maurice --- .../user-defined-functions/reduce/examples.md | 142 ++++++++++++ .../user-defined-functions/reduce/reduce.md | 219 ++++++------------ mkdocs.yml | 1 + 3 files changed, 215 insertions(+), 147 deletions(-) create mode 100644 docs/user-guide/user-defined-functions/reduce/examples.md diff --git a/docs/user-guide/user-defined-functions/reduce/examples.md b/docs/user-guide/user-defined-functions/reduce/examples.md new file mode 100644 index 0000000000..8cd86f2267 --- /dev/null +++ b/docs/user-guide/user-defined-functions/reduce/examples.md @@ -0,0 +1,142 @@ +# Reduce Examples + +## Prerequisites + +Install the ISB + +```shell +kubectl apply -f https://raw.githubusercontent.com/numaproj/numaflow/stable/examples/0-isbsvc-jetstream.yaml +``` + +## sum pipeline using fixed window +This is a simple reduce pipeline that just does summation (sum of numbers) but uses fixed window. +The snippet for the reduce vertex is as follows. + +![plot](../../../assets/simple-reduce.png) + +```yaml +- name: compute-sum + udf: + container: + # compute the sum + image: quay.io/numaio/numaflow-go/reduce-sum + groupBy: + window: + fixed: + length: 60s + keyed: true +``` + +[6-reduce-fixed-window.yaml](https://github.com/numaproj/numaflow/blob/main/examples/6-reduce-fixed-window.yaml) +has the complete pipeline definition. + +In this example we use a `parallelism` of `2`. We are setting a parallelism > 1 because it is a +keyed window. + +```yaml +- from: atoi + to: compute-sum + parallelism: 2 +``` + +```shell +kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/6-reduce-fixed-window.yaml +``` + +Output : +```text +2023/01/05 11:54:41 (sink) Payload - 300 Key - odd Start - 60000 End - 120000 +2023/01/05 11:54:41 (sink) Payload - 600 Key - even Start - 60000 End - 120000 +2023/01/05 11:54:41 (sink) Payload - 300 Key - odd Start - 120000 End - 180000 +2023/01/05 11:54:41 (sink) Payload - 600 Key - even Start - 120000 End - 180000 +2023/01/05 11:54:42 (sink) Payload - 600 Key - even Start - 180000 End - 240000 +2023/01/05 11:54:42 (sink) Payload - 300 Key - odd Start - 180000 End - 240000 +``` + +In our example, input is an HTTP source producing 2 messages each second with values 5 and 10, +and the event time starts from 60000. Since we have considered a fixed window of length 60s, +and also we are producing two messages with different keys "even" and "odd". Numaflow will create +two different windows with a start time of 60000 and an end time of 120000. So the output will be +300(5 * 60) and 600(10 * 60). + +If we had used a non keyed window (`keyed: false`), we would have seen one single output with value +of 900(300 of odd + 600 of even) for each window. + +## sum pipeline using sliding window +This is a simple reduce pipeline that just does summation (sum of numbers) but uses sliding window. +The snippet for the reduce vertex is as follows. + +![plot](../../../assets/simple-reduce.png) + +```yaml +- name: compute-sum + udf: + container: + # compute the sum + image: quay.io/numaio/numaflow-go/reduce-sum + groupBy: + window: + sliding: + length: 10s + slide: 5s + keyed: true +``` + +[7-reduce-sliding-window.yaml](https://github.com/numaproj/numaflow/blob/main/examples/examples/7-reduce-sliding-window.yaml) +has the complete pipeline definition + +```shell +kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/7-reduce-sliding-window.yaml +``` +Output: +```text +2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 60000 End - 120000 +2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 60000 End - 120000 +2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 70000 End - 130000 +2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 700000 End - 1300000 +2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 80000 End - 140000 +2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 80000 End - 140000 +``` + +In our example, input is an HTTP source producing 2 messages each second with values 5 and 10, +and the event time starts from 60000. Since we have considered a sliding window of length 60s +and slide 10s, and also we are producing two messages with different keys "even" and "odd". +Numaflow will create two different windows with a start time of 60000 and an end time of 120000, +and because the slide duration is 10s, a next set of windows will be created with start time of +70000 and an end time of 130000. Since its a sum operation the output will be 300(5 * 60) and 600(10 * 60). + +`Payload - 50 Key - odd Start - 10000 End - 70000`, we see 50 here for odd because the +first window has only 10 elements + +## complex reduce pipeline + +In the complex reduce example, we will +* chain of reduce functions +* use both fixed and sliding windows +* use keyed and non-keyed windowing + +![plot](../../../assets/complex-reduce.png) + +[8-reduce-complex-pipeline.yaml](https://github.com/numaproj/numaflow/blob/main/examples/examples/8-reduce-complex-pipeline.yaml) +has the complete pipeline definition + +```shell +kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/8-reduce-complex-pipeline.yaml +``` + +Output: +```text +2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 80000 End - 140000 +2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 90000 End - 150000 +2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 100000 End - 160000 +2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 110000 End - 170000 +2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 120000 End - 180000 +2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 130000 End - 190000 +``` + +In our example, first we have the reduce vertex with a fixed window of duration 5s. Since the input is 5 +and 10, the output from the first reduce vertex will be 25 (5 * 5) and 50 (5 * 10). This will be passed +to the next non-keyed reduce vertex with the fixed window duration of 10s. This being a non-keyed, it will +combine the inputs and produce the output of 150(25 * 2 + 50 * 2), which will be passed to the reduce +vertex with a sliding window of duration 60s and with the slide duration of 10s. Hence the final output +will be 900(150 * 6). \ No newline at end of file diff --git a/docs/user-guide/user-defined-functions/reduce/reduce.md b/docs/user-guide/user-defined-functions/reduce/reduce.md index d71418b8b2..1f4da8e42d 100644 --- a/docs/user-guide/user-defined-functions/reduce/reduce.md +++ b/docs/user-guide/user-defined-functions/reduce/reduce.md @@ -10,28 +10,38 @@ unbounded stream of data, we need a way to group elements into finite chunks usi chunks the reduce function is applied to the set of records produced using the concept of [windowing](./windowing/windowing.md). Unlike in _map_ vertex where only an element is given to user-defined function, in _reduce_ since -there is a group of elements, an iterator is passed to the reduce function. +there is a group of elements, an iterator is passed to the reduce function. The following is a generic +outlook of a reduce function. I have written the pseudo-code using the accumulator to show that very +powerful functions can be applied using this reduce semantics. ```python -# counter counts the number of elements in the array -# e.g. use-case of counter is counting number of users online per minute -def counter(key: str, datums: Iterator[Datum], md: Metadata) -> Messages: - counter = 0 - for _ in datums: - counter += 1 - return Messages(Message.to_vtx(key, str.encode(msg))) +# reduceFn func is a generic reduce function that processes a set of elements +def reduceFn(key: str, datums: Iterator[Datum], md: Metadata) -> Messages: + # initialize_accumalor could be any function that starts of with an empty + # state. eg, accumulator = 0 + accumulator = initialize_accumalor() + # we are iterating on the input set of elements + for d in datums: + # accumulator.add_input() can be any function. + # e.g., it could be as simple as accumulator += 1 + accumulator.add_input(d) + # once we are done with iterating on the elements, we return the result + # acumulator.result() can be str.encode(accumulator) + return Messages(Message.to_vtx(key, acumulator.result())) ``` The structure for defining a reduce vertex is as follows. ```yaml - - name: my-reduce-udf - udf: - container: - image: my-reduce-udf:latest - groupBy: - window: - ... - keyed: ... +- name: my-reduce-udf + udf: + container: + image: my-reduce-udf:latest + groupBy: + window: + ... + keyed: ... + storage: + ... ``` The reduce spec adds a new section called `groupBy` and this how we differentiate a _map_ vertex @@ -43,153 +53,68 @@ The reduce supports a parallelism value while defining the edge. This is because not supported in reduce vertex. If `parallelism` is not defined default of one will be used. ```yaml - - from: edge1 - to: my-reduce-reduce - parallelism: integer +- from: edge1 + to: my-reduce-reduce + parallelism: integer ``` It is wrong to give a `parallelism` > `1` if it is a _non-keyed_ vertex (`keyed: false`). +There are a couple of [examples](examples.md) that demonstrates Fixed windows, Sliding windows, +chaining of windows, keyed streams, etc. -# Examples +## Storage -## Prerequisites - -Install the ISB - -```shell -kubectl apply -f https://raw.githubusercontent.com/numaproj/numaflow/stable/examples/0-isbsvc-jetstream.yaml -``` - -## sum pipeline using fixed window -This is a simple reduce pipeline that just does summation (sum of numbers) but uses fixed window. -The snippet for the reduce vertex is as follows. - -![plot](../../../assets/simple-reduce.png) +Reduce unlike map requires persistence. To support persistence user has to define the +`storage` configuration. We replay the data stored in this storage on pod startup if there has +been a restart of the reduce pod caused due to pod migrations, etc. ```yaml - - name: compute-sum - udf: - container: - # compute the sum - image: quay.io/numaio/numaflow-go/reduce-sum - groupBy: - window: - fixed: - length: 60s - keyed: true +vertices: + - name: my-udf + udf: + groupBy: + storage: + .... ``` -[6-reduce-fixed-window.yaml](https://github.com/numaproj/numaflow/blob/main/examples/6-reduce-fixed-window.yaml) -has the complete pipeline definition. +### Persistent Volume Claim (PVC) -In this example we use a `parallelism` of `2`. We are setting a parallelism > 1 because it is a -keyed window. +`persistentVolumeClaim` supports the following fields, `volumeSize`, `storageClassName`, and`accessMode`. +As name suggests,`volumeSize` specifies the size of the volume. `accessMode` can be of many types, but for +reduce usecase we need only `ReadWriteOnce`. `storageClassName` can also be provided, more info on storage class +can be found [here](https://kubernetes.io/docs/concepts/storage/persistent-volumes#class-1). The default +value of `storageClassName` is `default` which is default StorageClass may be deployed to a Kubernetes +cluster by addon manager during installation. +#### Example ```yaml - - from: atoi - to: compute-sum - parallelism: 2 +vertices: + - name: my-udf + udf: + groupBy: + storage: + persistentVolumeClaim: + volumeSize: 10Gi + accessMode: ReadWriteOnce ``` -```shell -kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/6-reduce-fixed-window.yaml -``` +### EmptyDir -Output : -```text -2023/01/05 11:54:41 (sink) Payload - 300 Key - odd Start - 60000 End - 120000 -2023/01/05 11:54:41 (sink) Payload - 600 Key - even Start - 60000 End - 120000 -2023/01/05 11:54:41 (sink) Payload - 300 Key - odd Start - 120000 End - 180000 -2023/01/05 11:54:41 (sink) Payload - 600 Key - even Start - 120000 End - 180000 -2023/01/05 11:54:42 (sink) Payload - 600 Key - even Start - 180000 End - 240000 -2023/01/05 11:54:42 (sink) Payload - 300 Key - odd Start - 180000 End - 240000 -``` +We also support `emptyDir` for quick experimentation. We do not recommend this in production +setup. If we use `emptyDir`, we will end up in data loss if there are pod migrations. `emptyDir` +also takes an optional `sizeLimit`. `medium` field controls where emptyDir volumes are stored. +By default emptyDir volumes are stored on whatever medium that backs the node such as disk, SSD, +or network storage, depending on your environment. If you set the `medium` field to `"Memory"`, +Kubernetes mounts a tmpfs (RAM-backed filesystem) for you instead. -In our example, input is an HTTP source producing 2 messages each second with values 5 and 10, -and the event time starts from 60000. Since we have considered a fixed window of length 60s, -and also we are producing two messages with different keys "even" and "odd". Numaflow will create -two different windows with a start time of 60000 and an end time of 120000. So the output will be -300(5 * 60) and 600(10 * 60). - -If we had used a non keyed window (`keyed: false`), we would have seen one single output with value -of 900(300 of odd + 600 of even) for each window. - -## sum pipeline using sliding window -This is a simple reduce pipeline that just does summation (sum of numbers) but uses sliding window. -The snippet for the reduce vertex is as follows. - -![plot](../../../assets/simple-reduce.png) +#### Example ```yaml - - name: compute-sum - udf: - container: - # compute the sum - image: quay.io/numaio/numaflow-go/reduce-sum - groupBy: - window: - sliding: - length: 10s - slide: 5s - keyed: true -``` - -[7-reduce-sliding-window.yaml](https://github.com/numaproj/numaflow/blob/main/examples/examples/7-reduce-sliding-window.yaml) -has the complete pipeline definition - -```shell -kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/7-reduce-sliding-window.yaml -``` -Output: -```text -2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 60000 End - 120000 -2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 60000 End - 120000 -2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 70000 End - 130000 -2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 700000 End - 1300000 -2023/01/05 15:13:16 (sink) Payload - 300 Key - odd Start - 80000 End - 140000 -2023/01/05 15:13:16 (sink) Payload - 600 Key - even Start - 80000 End - 140000 -``` - -In our example, input is an HTTP source producing 2 messages each second with values 5 and 10, -and the event time starts from 60000. Since we have considered a sliding window of length 60s -and slide 10s, and also we are producing two messages with different keys "even" and "odd". -Numaflow will create two different windows with a start time of 60000 and an end time of 120000, -and because the slide duration is 10s, a next set of windows will be created with start time of -70000 and an end time of 130000. Since its a sum operation the output will be 300(5 * 60) and 600(10 * 60). - -`Payload - 50 Key - odd Start - 10000 End - 70000`, we see 50 here for odd because the -first window has only 10 elements - -## complex reduce pipeline - -In the complex reduce example, we will -* chain of reduce functions -* use both fixed and sliding windows -* use keyed and non-keyed windowing - -![plot](../../../assets/complex-reduce.png) - -[7-reduce-sliding-window.yaml](https://github.com/numaproj/numaflow/blob/main/examples/examples/8-reduce-complex-pipeline.yaml) -has the complete pipeline definition - -```shell -kubectl apply -f https://github.com/numaproj/numaflow/blob/main/examples/examples/8-reduce-complex-pipeline.yaml -``` - -Output: -```text -2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 80000 End - 140000 -2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 90000 End - 150000 -2023/01/05 15:33:55 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 100000 End - 160000 -2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 110000 End - 170000 -2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 120000 End - 180000 -2023/01/05 15:33:56 (sink) Payload - 900 Key - NON_KEYED_STREAM Start - 130000 End - 190000 -``` - -In our example, First we have the reduce vertex with a fixed window of duration 5s, Since the input is 5 -and 10, the output from the first reduce vertex will be 25 and 50, and this will be passed to the next -non keyed reduce vertex with the fixed window duration of 10s, since its non keyed it will combine the -inputs and produce the output of 150(25 * 2 + 50 * 2), which will be passed to the reduce vertex with -a sliding window of duration 60s and with the slide duration of 10s, Hence the final output will be -900(150 * 6). \ No newline at end of file +vertices: + - name: my-udf + udf: + groupBy: + storage: + emptyDir: {} +``` \ No newline at end of file diff --git a/mkdocs.yml b/mkdocs.yml index e58bdf559c..201bf384c2 100644 --- a/mkdocs.yml +++ b/mkdocs.yml @@ -68,6 +68,7 @@ nav: - Overview: "user-guide/user-defined-functions/reduce/windowing/windowing.md" - Fixed: "user-guide/user-defined-functions/reduce/windowing/fixed.md" - Sliding: "user-guide/user-defined-functions/reduce/windowing/sliding.md" + - Examples: "user-guide/user-defined-functions/reduce/examples.md" - user-guide/pipeline-tuning.md - user-guide/conditional-forwarding.md - user-guide/autoscaling.md From 3fcee6572470cf2a563b3f1dd37b3832ff5cb59b Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Wed, 11 Jan 2023 12:14:26 -0800 Subject: [PATCH 02/13] chore: move forward to top-level (#461) --- pkg/{isb => }/forward/forward.go | 0 pkg/{isb => }/forward/forward_test.go | 6 ++++-- pkg/{isb => }/forward/interfaces.go | 0 pkg/{isb => }/forward/metrics.go | 0 pkg/{isb => }/forward/options.go | 0 pkg/{isb => }/forward/shutdown.go | 0 pkg/{isb => }/forward/shutdown_test.go | 3 ++- pkg/isb/stores/jetstream/writer_test.go | 2 +- pkg/isb/stores/redis/read_test.go | 10 +++++----- pkg/isb/stores/redis/write_test.go | 2 +- pkg/reduce/pnf/processandforward.go | 2 +- pkg/reduce/readloop/ordered.go | 2 +- pkg/reduce/readloop/readloop.go | 2 +- pkg/reduce/reduce.go | 2 +- pkg/sinks/blackhole/blackhole.go | 10 +++++----- pkg/sinks/blackhole/blackhole_test.go | 7 ++++--- pkg/sinks/kafka/kafka.go | 13 +++++++------ pkg/sinks/kafka/kafka_test.go | 9 +++++---- pkg/sinks/logger/log.go | 10 +++++----- pkg/sinks/logger/log_test.go | 10 ++++++---- pkg/sinks/sinker.go | 2 +- pkg/sinks/udsink/sink.go | 23 ++++++++++++----------- pkg/sources/generator/tickgen.go | 10 +++++----- pkg/sources/http/http.go | 10 +++++----- pkg/sources/kafka/reader.go | 10 +++++----- pkg/sources/nats/nats.go | 10 +++++----- pkg/sources/sourcer.go | 2 +- pkg/udf/map_udf.go | 12 ++++++------ pkg/udf/reduce_udf.go | 5 +++-- 29 files changed, 92 insertions(+), 82 deletions(-) rename pkg/{isb => }/forward/forward.go (100%) rename pkg/{isb => }/forward/forward_test.go (99%) rename pkg/{isb => }/forward/interfaces.go (100%) rename pkg/{isb => }/forward/metrics.go (100%) rename pkg/{isb => }/forward/options.go (100%) rename pkg/{isb => }/forward/shutdown.go (100%) rename pkg/{isb => }/forward/shutdown_test.go (99%) diff --git a/pkg/isb/forward/forward.go b/pkg/forward/forward.go similarity index 100% rename from pkg/isb/forward/forward.go rename to pkg/forward/forward.go diff --git a/pkg/isb/forward/forward_test.go b/pkg/forward/forward_test.go similarity index 99% rename from pkg/isb/forward/forward_test.go rename to pkg/forward/forward_test.go index 8082544a7d..50dd3854a8 100644 --- a/pkg/isb/forward/forward_test.go +++ b/pkg/forward/forward_test.go @@ -26,15 +26,17 @@ import ( "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/watermark/generic" - dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/prometheus/client_golang/prometheus/testutil" + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/shared/logging" + "github.com/stretchr/testify/assert" + "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" udfapplier "github.com/numaproj/numaflow/pkg/udf/function" - "github.com/stretchr/testify/assert" ) var ( diff --git a/pkg/isb/forward/interfaces.go b/pkg/forward/interfaces.go similarity index 100% rename from pkg/isb/forward/interfaces.go rename to pkg/forward/interfaces.go diff --git a/pkg/isb/forward/metrics.go b/pkg/forward/metrics.go similarity index 100% rename from pkg/isb/forward/metrics.go rename to pkg/forward/metrics.go diff --git a/pkg/isb/forward/options.go b/pkg/forward/options.go similarity index 100% rename from pkg/isb/forward/options.go rename to pkg/forward/options.go diff --git a/pkg/isb/forward/shutdown.go b/pkg/forward/shutdown.go similarity index 100% rename from pkg/isb/forward/shutdown.go rename to pkg/forward/shutdown.go diff --git a/pkg/isb/forward/shutdown_test.go b/pkg/forward/shutdown_test.go similarity index 99% rename from pkg/isb/forward/shutdown_test.go rename to pkg/forward/shutdown_test.go index a3b688840d..b6c06e9229 100644 --- a/pkg/isb/forward/shutdown_test.go +++ b/pkg/forward/shutdown_test.go @@ -26,9 +26,10 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/stretchr/testify/assert" + "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" - "github.com/stretchr/testify/assert" ) type myShutdownTest struct { diff --git a/pkg/isb/stores/jetstream/writer_test.go b/pkg/isb/stores/jetstream/writer_test.go index 4485825bd6..61cbb2b02d 100644 --- a/pkg/isb/stores/jetstream/writer_test.go +++ b/pkg/isb/stores/jetstream/writer_test.go @@ -21,13 +21,13 @@ import ( "testing" "time" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/stretchr/testify/assert" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/isb/testutils" natstest "github.com/numaproj/numaflow/pkg/shared/clients/nats/test" ) diff --git a/pkg/isb/stores/redis/read_test.go b/pkg/isb/stores/redis/read_test.go index c3c69c3b83..8baef1d587 100644 --- a/pkg/isb/stores/redis/read_test.go +++ b/pkg/isb/stores/redis/read_test.go @@ -26,6 +26,7 @@ import ( "testing" "time" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/go-redis/redis/v8" @@ -35,7 +36,6 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/isb/testutils" redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis" ) @@ -132,7 +132,7 @@ func TestRedisCheckBacklog(t *testing.T) { "to1": rqw, } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, forward.WithReadBatchSize(10)) + f, err := forward2.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, forward2.WithReadBatchSize(10)) stopped := f.Start() // validate the length of the toStep stream. @@ -283,7 +283,7 @@ type ReadWritePerformance struct { rclient *redisclient.RedisClient rqr *BufferRead rqw *BufferWrite - isdf *forward.InterStepDataForward + isdf *forward2.InterStepDataForward count int64 withPipelining bool cancel context.CancelFunc @@ -324,7 +324,7 @@ func (suite *ReadWritePerformance) SetupSuite() { }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - isdf, _ := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) + isdf, _ := forward2.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) suite.ctx = ctx suite.rclient = client @@ -412,7 +412,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatencyPipelining() { "to1": suite.rqw, } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - suite.isdf, _ = forward.NewInterStepDataForward(vertex, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) + suite.isdf, _ = forward2.NewInterStepDataForward(vertex, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) diff --git a/pkg/isb/stores/redis/write_test.go b/pkg/isb/stores/redis/write_test.go index b4160873aa..c9bb1a4cd1 100644 --- a/pkg/isb/stores/redis/write_test.go +++ b/pkg/isb/stores/redis/write_test.go @@ -24,6 +24,7 @@ import ( "testing" "time" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/go-redis/redis/v8" @@ -31,7 +32,6 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/isb/testutils" redisclient "github.com/numaproj/numaflow/pkg/shared/clients/redis" ) diff --git a/pkg/reduce/pnf/processandforward.go b/pkg/reduce/pnf/processandforward.go index 8e196ee748..0b9b708eb3 100644 --- a/pkg/reduce/pnf/processandforward.go +++ b/pkg/reduce/pnf/processandforward.go @@ -27,6 +27,7 @@ import ( "sync" "time" + "github.com/numaproj/numaflow/pkg/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" @@ -35,7 +36,6 @@ import ( "k8s.io/apimachinery/pkg/util/wait" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" "github.com/numaproj/numaflow/pkg/watermark/processor" diff --git a/pkg/reduce/readloop/ordered.go b/pkg/reduce/readloop/ordered.go index 3f2a818dc4..6a996440dc 100644 --- a/pkg/reduce/readloop/ordered.go +++ b/pkg/reduce/readloop/ordered.go @@ -24,12 +24,12 @@ import ( "go.uber.org/zap" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/watermark/publish" "github.com/numaproj/numaflow/pkg/reduce/pnf" diff --git a/pkg/reduce/readloop/readloop.go b/pkg/reduce/readloop/readloop.go index 01a6e5372c..be3b1d6a31 100644 --- a/pkg/reduce/readloop/readloop.go +++ b/pkg/reduce/readloop/readloop.go @@ -36,12 +36,12 @@ import ( "go.uber.org/zap" "k8s.io/apimachinery/pkg/util/wait" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/metrics" pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/processor" diff --git a/pkg/reduce/reduce.go b/pkg/reduce/reduce.go index c49228c720..efe52fad61 100644 --- a/pkg/reduce/reduce.go +++ b/pkg/reduce/reduce.go @@ -23,6 +23,7 @@ import ( "context" "time" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq" @@ -30,7 +31,6 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/reduce/readloop" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" diff --git a/pkg/sinks/blackhole/blackhole.go b/pkg/sinks/blackhole/blackhole.go index bac908f311..39b2395fe3 100644 --- a/pkg/sinks/blackhole/blackhole.go +++ b/pkg/sinks/blackhole/blackhole.go @@ -19,6 +19,7 @@ package blackhole import ( "context" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" @@ -29,7 +30,6 @@ import ( "go.uber.org/zap" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" ) @@ -37,7 +37,7 @@ import ( type Blackhole struct { name string pipelineName string - isdf *forward.InterStepDataForward + isdf *forward2.InterStepDataForward logger *zap.SugaredLogger } @@ -66,14 +66,14 @@ func NewBlackhole(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWaterma bh.logger = logging.NewLogger() } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(bh.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(bh.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: bh}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: bh}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/blackhole/blackhole_test.go b/pkg/sinks/blackhole/blackhole_test.go index ccf3b87e71..75ef4af8f0 100644 --- a/pkg/sinks/blackhole/blackhole_test.go +++ b/pkg/sinks/blackhole/blackhole_test.go @@ -21,16 +21,17 @@ import ( "testing" "time" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/watermark/generic" + "github.com/stretchr/testify/assert" + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/isb/testutils" - "github.com/stretchr/testify/assert" ) var ( @@ -119,7 +120,7 @@ func TestBlackhole_ForwardToTwoVertex(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) + f, err := forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/sinks/kafka/kafka.go b/pkg/sinks/kafka/kafka.go index 718b2b938e..9dc9c2c42d 100644 --- a/pkg/sinks/kafka/kafka.go +++ b/pkg/sinks/kafka/kafka.go @@ -22,14 +22,15 @@ import ( "time" "github.com/Shopify/sarama" + "go.uber.org/zap" + + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/publish" - "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/shared/util" @@ -42,7 +43,7 @@ type ToKafka struct { producer sarama.AsyncProducer connected bool topic string - isdf *forward.InterStepDataForward + isdf *forward2.InterStepDataForward kafkaSink *dfv1.KafkaSink log *zap.SugaredLogger } @@ -77,14 +78,14 @@ func NewToKafka(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWatermark toKafka.topic = kafkaSink.Topic toKafka.kafkaSink = kafkaSink - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(toKafka.log)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(toKafka.log)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - f, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + f, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/kafka/kafka_test.go b/pkg/sinks/kafka/kafka_test.go index e839131212..4fe178e9e9 100644 --- a/pkg/sinks/kafka/kafka_test.go +++ b/pkg/sinks/kafka/kafka_test.go @@ -19,9 +19,11 @@ package kafka import ( "context" "fmt" - "github.com/numaproj/numaflow/pkg/udf/applier" "testing" + forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/udf/applier" + "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/watermark/generic" @@ -31,7 +33,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/shared/logging" ) @@ -49,7 +50,7 @@ func TestWriteSuccessToKafka(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromEdgeList(generic.GetBufferNameList(vertex.GetToBuffers())) - toKafka.isdf, err = forward.NewInterStepDataForward(vertex, fromStep, map[string]isb.BufferWriter{"name": toKafka}, forward.All, applier.Terminal, fetchWatermark, publishWatermark) + toKafka.isdf, err = forward2.NewInterStepDataForward(vertex, fromStep, map[string]isb.BufferWriter{"name": toKafka}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) toKafka.kafkaSink = vertex.Spec.Sink.Kafka toKafka.name = "Test" @@ -102,7 +103,7 @@ func TestWriteFailureToKafka(t *testing.T) { }} toSteps := map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - toKafka.isdf, err = forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) + toKafka.isdf, err = forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) toKafka.name = "Test" toKafka.topic = "topic-1" diff --git a/pkg/sinks/logger/log.go b/pkg/sinks/logger/log.go index 0383d25ede..09ab0cf7e6 100644 --- a/pkg/sinks/logger/log.go +++ b/pkg/sinks/logger/log.go @@ -20,6 +20,7 @@ import ( "context" "log" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/publish" @@ -29,7 +30,6 @@ import ( "go.uber.org/zap" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" ) @@ -38,7 +38,7 @@ import ( type ToLog struct { name string pipelineName string - isdf *forward.InterStepDataForward + isdf *forward2.InterStepDataForward logger *zap.SugaredLogger } @@ -67,14 +67,14 @@ func NewToLog(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWatermark f toLog.logger = logging.NewLogger() } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(toLog.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(toLog.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toLog}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toLog}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/logger/log_test.go b/pkg/sinks/logger/log_test.go index 791d36a9a8..26d3370fc8 100644 --- a/pkg/sinks/logger/log_test.go +++ b/pkg/sinks/logger/log_test.go @@ -18,18 +18,20 @@ package logger import ( "context" - "github.com/numaproj/numaflow/pkg/udf/applier" "testing" "time" + forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/udf/applier" + "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/watermark/generic" + "github.com/stretchr/testify/assert" + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/isb/testutils" - "github.com/stretchr/testify/assert" ) var ( @@ -125,7 +127,7 @@ func TestToLog_ForwardToTwoVertex(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) + f, err := forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/sinks/sinker.go b/pkg/sinks/sinker.go index d2803f08ea..c95794a8e9 100644 --- a/pkg/sinks/sinker.go +++ b/pkg/sinks/sinker.go @@ -17,8 +17,8 @@ limitations under the License. package sinks import ( + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" ) // Sinker interface defines what a Sink should implement. diff --git a/pkg/sinks/udsink/sink.go b/pkg/sinks/udsink/sink.go index 56bfba8289..ac01c498e2 100644 --- a/pkg/sinks/udsink/sink.go +++ b/pkg/sinks/udsink/sink.go @@ -22,21 +22,22 @@ import ( "time" sinkpb "github.com/numaproj/numaflow-go/pkg/apis/proto/sink/v1" + "go.uber.org/zap" + "google.golang.org/protobuf/types/known/timestamppb" + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/publish" - "go.uber.org/zap" - "google.golang.org/protobuf/types/known/timestamppb" ) type UserDefinedSink struct { name string pipelineName string - isdf *forward.InterStepDataForward + isdf *forward2.InterStepDataForward logger *zap.SugaredLogger udsink *udsGRPCBasedUDSink } @@ -65,10 +66,10 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchW s.logger = logging.NewLogger() } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(s.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(s.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } udsink, err := NewUDSGRPCBasedUDSink() @@ -76,7 +77,7 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchW return nil, fmt.Errorf("failed to create gRPC client, %w", err) } s.udsink = udsink - isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: s}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: s}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } @@ -108,9 +109,9 @@ func (s *UserDefinedSink) Write(ctx context.Context, messages []isb.Message) ([] return nil, s.udsink.Apply(ctx, msgs) } -func (br *UserDefinedSink) Close() error { - if br.udsink != nil { - return br.udsink.CloseConn(context.Background()) +func (s *UserDefinedSink) Close() error { + if s.udsink != nil { + return s.udsink.CloseConn(context.Background()) } return nil } @@ -120,7 +121,7 @@ func (s *UserDefinedSink) Start() <-chan struct{} { ctx, cancel := context.WithTimeout(context.Background(), 60*time.Second) defer cancel() if err := s.udsink.WaitUntilReady(ctx); err != nil { - s.logger.Fatalf("failed on UDSink readiness check, %w", err) + s.logger.Fatalf("failed on UDSink readiness check, %s", err) } return s.isdf.Start() } diff --git a/pkg/sources/generator/tickgen.go b/pkg/sources/generator/tickgen.go index c17f0e9036..50f1087725 100644 --- a/pkg/sources/generator/tickgen.go +++ b/pkg/sources/generator/tickgen.go @@ -30,8 +30,8 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" @@ -91,7 +91,7 @@ type memgen struct { // once terminated the source will not generate any more records. cancel context.CancelFunc // forwarder to read from the source and write to the interstep buffer. - forwarder *forward.InterStepDataForward + forwarder *forward2.InterStepDataForward // lifecycleCtx context is used to control the lifecycle of this instance. lifecycleCtx context.Context // read timeout for the reader @@ -180,10 +180,10 @@ func NewMemGen(vertexInstance *dfv1.VertexInstance, destinations[w.GetName()] = w } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(gensrc.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(gensrc.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } @@ -191,7 +191,7 @@ func NewMemGen(vertexInstance *dfv1.VertexInstance, gensrc.sourcePublishWM = gensrc.buildSourceWatermarkPublisher(publishWMStores) // we pass in the context to forwarder as well so that it can shut down when we cancel the context - forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, gensrc, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, gensrc, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sources/http/http.go b/pkg/sources/http/http.go index f8f29c4638..5f1c9f8717 100644 --- a/pkg/sources/http/http.go +++ b/pkg/sources/http/http.go @@ -29,8 +29,8 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedtls "github.com/numaproj/numaflow/pkg/shared/tls" @@ -51,7 +51,7 @@ type httpSource struct { messages chan *isb.ReadMessage logger *zap.SugaredLogger - forwarder *forward.InterStepDataForward + forwarder *forward2.InterStepDataForward // source watermark publisher sourcePublishWM publish.Publisher // context cancel function @@ -186,13 +186,13 @@ func New(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWriter, fetchW destinations[w.GetName()] = w } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(h.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(h.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, h, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, h, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { h.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/kafka/reader.go b/pkg/sources/kafka/reader.go index adfa7726c3..99ce3bd58e 100644 --- a/pkg/sources/kafka/reader.go +++ b/pkg/sources/kafka/reader.go @@ -29,8 +29,8 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" @@ -53,7 +53,7 @@ type KafkaSource struct { // kafka brokers brokers []string // forwarder that writes the consumed data to destination - forwarder *forward.InterStepDataForward + forwarder *forward2.InterStepDataForward // context cancel function cancelfn context.CancelFunc // lifecycle context @@ -322,13 +322,13 @@ func NewKafkaSource(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWri destinations[w.GetName()] = w } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(kafkasource.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(kafkasource.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, kafkasource, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, kafkasource, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { kafkasource.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/nats/nats.go b/pkg/sources/nats/nats.go index 5dca00e858..81405a3de1 100644 --- a/pkg/sources/nats/nats.go +++ b/pkg/sources/nats/nats.go @@ -26,8 +26,8 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" @@ -51,7 +51,7 @@ type natsSource struct { readTimeout time.Duration cancelfn context.CancelFunc - forwarder *forward.InterStepDataForward + forwarder *forward2.InterStepDataForward // source watermark publisher sourcePublishWM publish.Publisher } @@ -78,13 +78,13 @@ func New(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWriter, fetchW for _, w := range writers { destinations[w.GetName()] = w } - forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(n.logger)} + forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(n.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, n, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, n, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { n.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/sourcer.go b/pkg/sources/sourcer.go index 6ba242a309..b712825c16 100644 --- a/pkg/sources/sourcer.go +++ b/pkg/sources/sourcer.go @@ -17,8 +17,8 @@ limitations under the License. package sources import ( + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" ) // Sourcer interface provides an isb.BufferReader abstraction over the underlying data source. diff --git a/pkg/udf/map_udf.go b/pkg/udf/map_udf.go index dcefba2358..599cc2b279 100644 --- a/pkg/udf/map_udf.go +++ b/pkg/udf/map_udf.go @@ -25,8 +25,8 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + forward2 "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" @@ -82,7 +82,7 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { } } - conditionalForwarder := forward.GoWhere(func(key string) ([]string, error) { + conditionalForwarder := forward2.GoWhere(func(key string) ([]string, error) { result := []string{} if key == dfv1.MessageKeyDrop { return result, nil @@ -117,15 +117,15 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { }() log.Infow("Start processing udf messages", zap.String("isbsvc", string(u.ISBSvcType)), zap.String("from", fromBufferName), zap.Any("to", toBuffers)) - opts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeMapUDF), forward.WithLogger(log)} + opts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeMapUDF), forward2.WithLogger(log)} if x := u.VertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - opts = append(opts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) - opts = append(opts, forward.WithUDFConcurrency(int(*x.ReadBatchSize))) + opts = append(opts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + opts = append(opts, forward2.WithUDFConcurrency(int(*x.ReadBatchSize))) } } - forwarder, err := forward.NewInterStepDataForward(u.VertexInstance.Vertex, reader, writers, conditionalForwarder, udfHandler, fetchWatermark, publishWatermark, opts...) + forwarder, err := forward2.NewInterStepDataForward(u.VertexInstance.Vertex, reader, writers, conditionalForwarder, udfHandler, fetchWatermark, publishWatermark, opts...) if err != nil { return err } diff --git a/pkg/udf/reduce_udf.go b/pkg/udf/reduce_udf.go index 5cb5572160..3f128fee1d 100644 --- a/pkg/udf/reduce_udf.go +++ b/pkg/udf/reduce_udf.go @@ -23,16 +23,17 @@ import ( "sync" "time" - "github.com/numaproj/numaflow/pkg/reduce/pbq/store/wal" "go.uber.org/zap" + "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/wal" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/window/strategy/fixed" "github.com/numaproj/numaflow/pkg/window/strategy/sliding" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/isb/forward" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce" "github.com/numaproj/numaflow/pkg/shared/logging" From 2298615374ff699179384627cfe6e5b13da793af Mon Sep 17 00:00:00 2001 From: Juanlu Yu <19543684+chromevoid@users.noreply.github.com> Date: Wed, 11 Jan 2023 14:06:32 -0800 Subject: [PATCH 03/13] fix: JetStream context KV store/watch fix (#460) Signed-off-by: jyu6 --- pkg/isb/stores/redis/read_test.go | 14 ++-- .../nats/in_cluster_jetstream_client.go | 2 +- pkg/shared/clients/nats/interface.go | 2 +- pkg/sinks/blackhole/blackhole.go | 10 +-- pkg/sinks/blackhole/blackhole_test.go | 4 +- pkg/sinks/kafka/kafka.go | 14 ++-- pkg/sinks/kafka/kafka_test.go | 6 +- pkg/sinks/logger/log.go | 10 +-- pkg/sinks/logger/log_test.go | 4 +- pkg/sinks/udsink/sink.go | 10 +-- pkg/sources/generator/tickgen.go | 10 +-- pkg/sources/http/http.go | 10 +-- pkg/sources/kafka/reader.go | 10 +-- pkg/sources/nats/nats.go | 10 +-- pkg/udf/map_udf.go | 12 +-- pkg/watermark/store/jetstream/kv_store.go | 83 ++++++++++++------- pkg/watermark/store/jetstream/kv_watch.go | 75 ++++++++++------- 17 files changed, 161 insertions(+), 125 deletions(-) diff --git a/pkg/isb/stores/redis/read_test.go b/pkg/isb/stores/redis/read_test.go index 8baef1d587..57a0cdb1cd 100644 --- a/pkg/isb/stores/redis/read_test.go +++ b/pkg/isb/stores/redis/read_test.go @@ -26,7 +26,7 @@ import ( "testing" "time" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/go-redis/redis/v8" @@ -42,7 +42,7 @@ import ( var ( redisOptions = &redis.UniversalOptions{ - //Addrs: []string{":7000", ":7001", ":7002", ":7003", ":7004", ":7005"} + // Addrs: []string{":7000", ":7001", ":7002", ":7003", ":7004", ":7005"} Addrs: []string{":6379"}, } @@ -132,7 +132,7 @@ func TestRedisCheckBacklog(t *testing.T) { "to1": rqw, } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward2.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, forward2.WithReadBatchSize(10)) + f, err := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, forward.WithReadBatchSize(10)) stopped := f.Start() // validate the length of the toStep stream. @@ -283,7 +283,7 @@ type ReadWritePerformance struct { rclient *redisclient.RedisClient rqr *BufferRead rqw *BufferWrite - isdf *forward2.InterStepDataForward + isdf *forward.InterStepDataForward count int64 withPipelining bool cancel context.CancelFunc @@ -324,7 +324,7 @@ func (suite *ReadWritePerformance) SetupSuite() { }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - isdf, _ := forward2.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) + isdf, _ := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) suite.ctx = ctx suite.rclient = client @@ -412,7 +412,7 @@ func (suite *ReadWritePerformance) TestReadWriteLatencyPipelining() { "to1": suite.rqw, } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - suite.isdf, _ = forward2.NewInterStepDataForward(vertex, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) + suite.isdf, _ = forward.NewInterStepDataForward(vertex, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark) suite.False(suite.rqw.IsFull()) var writeMessages = make([]isb.Message, 0, suite.count) @@ -491,7 +491,7 @@ func generateLatencySlice(xMessages []redis.XMessage, suite *ReadWritePerformanc suite.NoError(err) // We store a difference of the id and the offset in the to stream. - //This gives us a difference between the time it was read that is stored in ID of the Header and the time it was written as stored in the ID. + // This gives us a difference between the time it was read that is stored in ID of the Header and the time it was written as stored in the ID. latency[idx] = float64(id - offset) } diff --git a/pkg/shared/clients/nats/in_cluster_jetstream_client.go b/pkg/shared/clients/nats/in_cluster_jetstream_client.go index 9004b6175d..87376d2dd5 100644 --- a/pkg/shared/clients/nats/in_cluster_jetstream_client.go +++ b/pkg/shared/clients/nats/in_cluster_jetstream_client.go @@ -65,7 +65,7 @@ func (isc *inClusterJetStreamClient) connect(ctx context.Context) (*nats.Conn, e return natsJetStreamConnection(ctx, url, natsOpts) } -// Connect is used to establish an incluster NATS jetstream connection +// Connect is used to establish an inCluster NATS JetStream connection func (isc *inClusterJetStreamClient) Connect(ctx context.Context, opts ...JetStreamClientOption) (*NatsConn, error) { options := defaultJetStreamClientOptions() for _, o := range opts { diff --git a/pkg/shared/clients/nats/interface.go b/pkg/shared/clients/nats/interface.go index 0c174f3e28..357e48b5fe 100644 --- a/pkg/shared/clients/nats/interface.go +++ b/pkg/shared/clients/nats/interface.go @@ -18,7 +18,7 @@ package nats import "context" -// JetStreamClient is used to provide a jetstream client +// JetStreamClient is used to provide a JetStream client type JetStreamClient interface { Connect(ctx context.Context, opts ...JetStreamClientOption) (*NatsConn, error) } diff --git a/pkg/sinks/blackhole/blackhole.go b/pkg/sinks/blackhole/blackhole.go index 39b2395fe3..cc5536382c 100644 --- a/pkg/sinks/blackhole/blackhole.go +++ b/pkg/sinks/blackhole/blackhole.go @@ -19,7 +19,7 @@ package blackhole import ( "context" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" @@ -37,7 +37,7 @@ import ( type Blackhole struct { name string pipelineName string - isdf *forward2.InterStepDataForward + isdf *forward.InterStepDataForward logger *zap.SugaredLogger } @@ -66,14 +66,14 @@ func NewBlackhole(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWaterma bh.logger = logging.NewLogger() } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(bh.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(bh.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: bh}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: bh}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/blackhole/blackhole_test.go b/pkg/sinks/blackhole/blackhole_test.go index 75ef4af8f0..d06b3e3f0f 100644 --- a/pkg/sinks/blackhole/blackhole_test.go +++ b/pkg/sinks/blackhole/blackhole_test.go @@ -21,7 +21,7 @@ import ( "testing" "time" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" @@ -120,7 +120,7 @@ func TestBlackhole_ForwardToTwoVertex(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) + f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/sinks/kafka/kafka.go b/pkg/sinks/kafka/kafka.go index 9dc9c2c42d..c6fdf75d82 100644 --- a/pkg/sinks/kafka/kafka.go +++ b/pkg/sinks/kafka/kafka.go @@ -24,7 +24,7 @@ import ( "github.com/Shopify/sarama" "go.uber.org/zap" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/publish" @@ -43,7 +43,7 @@ type ToKafka struct { producer sarama.AsyncProducer connected bool topic string - isdf *forward2.InterStepDataForward + isdf *forward.InterStepDataForward kafkaSink *dfv1.KafkaSink log *zap.SugaredLogger } @@ -61,14 +61,14 @@ func WithLogger(log *zap.SugaredLogger) Option { func NewToKafka(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWatermark fetch.Fetcher, publishWatermark map[string]publish.Publisher, opts ...Option) (*ToKafka, error) { kafkaSink := vertex.Spec.Sink.Kafka toKafka := new(ToKafka) - //apply options for kafka sink + // apply options for kafka sink for _, o := range opts { if err := o(toKafka); err != nil { return nil, err } } - //set default logger + // set default logger if toKafka.log == nil { toKafka.log = logging.NewLogger() } @@ -78,14 +78,14 @@ func NewToKafka(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWatermark toKafka.topic = kafkaSink.Topic toKafka.kafkaSink = kafkaSink - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(toKafka.log)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(toKafka.log)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - f, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + f, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/kafka/kafka_test.go b/pkg/sinks/kafka/kafka_test.go index 4fe178e9e9..a825ee9c82 100644 --- a/pkg/sinks/kafka/kafka_test.go +++ b/pkg/sinks/kafka/kafka_test.go @@ -21,7 +21,7 @@ import ( "fmt" "testing" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" @@ -50,7 +50,7 @@ func TestWriteSuccessToKafka(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromEdgeList(generic.GetBufferNameList(vertex.GetToBuffers())) - toKafka.isdf, err = forward2.NewInterStepDataForward(vertex, fromStep, map[string]isb.BufferWriter{"name": toKafka}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) + toKafka.isdf, err = forward.NewInterStepDataForward(vertex, fromStep, map[string]isb.BufferWriter{"name": toKafka}, forward.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) toKafka.kafkaSink = vertex.Spec.Sink.Kafka toKafka.name = "Test" @@ -103,7 +103,7 @@ func TestWriteFailureToKafka(t *testing.T) { }} toSteps := map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toKafka} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - toKafka.isdf, err = forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) + toKafka.isdf, err = forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) toKafka.name = "Test" toKafka.topic = "topic-1" diff --git a/pkg/sinks/logger/log.go b/pkg/sinks/logger/log.go index 09ab0cf7e6..e1cf6c0731 100644 --- a/pkg/sinks/logger/log.go +++ b/pkg/sinks/logger/log.go @@ -20,7 +20,7 @@ import ( "context" "log" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/publish" @@ -38,7 +38,7 @@ import ( type ToLog struct { name string pipelineName string - isdf *forward2.InterStepDataForward + isdf *forward.InterStepDataForward logger *zap.SugaredLogger } @@ -67,14 +67,14 @@ func NewToLog(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchWatermark f toLog.logger = logging.NewLogger() } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(toLog.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(toLog.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toLog}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: toLog}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/logger/log_test.go b/pkg/sinks/logger/log_test.go index 26d3370fc8..eea15a3a00 100644 --- a/pkg/sinks/logger/log_test.go +++ b/pkg/sinks/logger/log_test.go @@ -21,7 +21,7 @@ import ( "testing" "time" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" @@ -127,7 +127,7 @@ func TestToLog_ForwardToTwoVertex(t *testing.T) { }, }} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward2.NewInterStepDataForward(vertex, fromStep, toSteps, forward2.All, applier.Terminal, fetchWatermark, publishWatermark) + f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, forward.All, applier.Terminal, fetchWatermark, publishWatermark) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/sinks/udsink/sink.go b/pkg/sinks/udsink/sink.go index ac01c498e2..129f6eacd8 100644 --- a/pkg/sinks/udsink/sink.go +++ b/pkg/sinks/udsink/sink.go @@ -26,7 +26,7 @@ import ( "google.golang.org/protobuf/types/known/timestamppb" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" @@ -37,7 +37,7 @@ import ( type UserDefinedSink struct { name string pipelineName string - isdf *forward2.InterStepDataForward + isdf *forward.InterStepDataForward logger *zap.SugaredLogger udsink *udsGRPCBasedUDSink } @@ -66,10 +66,10 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchW s.logger = logging.NewLogger() } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSink), forward2.WithLogger(s.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSink), forward.WithLogger(s.logger)} if x := vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } udsink, err := NewUDSGRPCBasedUDSink() @@ -77,7 +77,7 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, fromBuffer isb.BufferReader, fetchW return nil, fmt.Errorf("failed to create gRPC client, %w", err) } s.udsink = udsink - isdf, err := forward2.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: s}, forward2.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) + isdf, err := forward.NewInterStepDataForward(vertex, fromBuffer, map[string]isb.BufferWriter{vertex.GetToBuffers()[0].Name: s}, forward.All, applier.Terminal, fetchWatermark, publishWatermark, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sources/generator/tickgen.go b/pkg/sources/generator/tickgen.go index 50f1087725..a941b2ea98 100644 --- a/pkg/sources/generator/tickgen.go +++ b/pkg/sources/generator/tickgen.go @@ -30,7 +30,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -91,7 +91,7 @@ type memgen struct { // once terminated the source will not generate any more records. cancel context.CancelFunc // forwarder to read from the source and write to the interstep buffer. - forwarder *forward2.InterStepDataForward + forwarder *forward.InterStepDataForward // lifecycleCtx context is used to control the lifecycle of this instance. lifecycleCtx context.Context // read timeout for the reader @@ -180,10 +180,10 @@ func NewMemGen(vertexInstance *dfv1.VertexInstance, destinations[w.GetName()] = w } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(gensrc.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(gensrc.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } @@ -191,7 +191,7 @@ func NewMemGen(vertexInstance *dfv1.VertexInstance, gensrc.sourcePublishWM = gensrc.buildSourceWatermarkPublisher(publishWMStores) // we pass in the context to forwarder as well so that it can shut down when we cancel the context - forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, gensrc, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, gensrc, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sources/http/http.go b/pkg/sources/http/http.go index 5f1c9f8717..f2f0f42af6 100644 --- a/pkg/sources/http/http.go +++ b/pkg/sources/http/http.go @@ -29,7 +29,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -51,7 +51,7 @@ type httpSource struct { messages chan *isb.ReadMessage logger *zap.SugaredLogger - forwarder *forward2.InterStepDataForward + forwarder *forward.InterStepDataForward // source watermark publisher sourcePublishWM publish.Publisher // context cancel function @@ -186,13 +186,13 @@ func New(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWriter, fetchW destinations[w.GetName()] = w } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(h.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(h.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, h, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, h, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { h.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/kafka/reader.go b/pkg/sources/kafka/reader.go index 99ce3bd58e..a6e9a1a4d3 100644 --- a/pkg/sources/kafka/reader.go +++ b/pkg/sources/kafka/reader.go @@ -29,7 +29,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -53,7 +53,7 @@ type KafkaSource struct { // kafka brokers brokers []string // forwarder that writes the consumed data to destination - forwarder *forward2.InterStepDataForward + forwarder *forward.InterStepDataForward // context cancel function cancelfn context.CancelFunc // lifecycle context @@ -322,13 +322,13 @@ func NewKafkaSource(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWri destinations[w.GetName()] = w } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(kafkasource.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(kafkasource.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, kafkasource, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, kafkasource, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { kafkasource.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/nats/nats.go b/pkg/sources/nats/nats.go index 81405a3de1..4fe059971b 100644 --- a/pkg/sources/nats/nats.go +++ b/pkg/sources/nats/nats.go @@ -26,7 +26,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" metricspkg "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -51,7 +51,7 @@ type natsSource struct { readTimeout time.Duration cancelfn context.CancelFunc - forwarder *forward2.InterStepDataForward + forwarder *forward.InterStepDataForward // source watermark publisher sourcePublishWM publish.Publisher } @@ -78,13 +78,13 @@ func New(vertexInstance *dfv1.VertexInstance, writers []isb.BufferWriter, fetchW for _, w := range writers { destinations[w.GetName()] = w } - forwardOpts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeSource), forward2.WithLogger(n.logger)} + forwardOpts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeSource), forward.WithLogger(n.logger)} if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - forwardOpts = append(forwardOpts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) + forwardOpts = append(forwardOpts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := forward2.NewInterStepDataForward(vertexInstance.Vertex, n, destinations, forward2.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) + forwarder, err := forward.NewInterStepDataForward(vertexInstance.Vertex, n, destinations, forward.All, applier.Terminal, fetchWM, publishWM, forwardOpts...) if err != nil { n.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/udf/map_udf.go b/pkg/udf/map_udf.go index 599cc2b279..84a32eb729 100644 --- a/pkg/udf/map_udf.go +++ b/pkg/udf/map_udf.go @@ -25,7 +25,7 @@ import ( "go.uber.org/zap" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - forward2 "github.com/numaproj/numaflow/pkg/forward" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" @@ -82,7 +82,7 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { } } - conditionalForwarder := forward2.GoWhere(func(key string) ([]string, error) { + conditionalForwarder := forward.GoWhere(func(key string) ([]string, error) { result := []string{} if key == dfv1.MessageKeyDrop { return result, nil @@ -117,15 +117,15 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { }() log.Infow("Start processing udf messages", zap.String("isbsvc", string(u.ISBSvcType)), zap.String("from", fromBufferName), zap.Any("to", toBuffers)) - opts := []forward2.Option{forward2.WithVertexType(dfv1.VertexTypeMapUDF), forward2.WithLogger(log)} + opts := []forward.Option{forward.WithVertexType(dfv1.VertexTypeMapUDF), forward.WithLogger(log)} if x := u.VertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { - opts = append(opts, forward2.WithReadBatchSize(int64(*x.ReadBatchSize))) - opts = append(opts, forward2.WithUDFConcurrency(int(*x.ReadBatchSize))) + opts = append(opts, forward.WithReadBatchSize(int64(*x.ReadBatchSize))) + opts = append(opts, forward.WithUDFConcurrency(int(*x.ReadBatchSize))) } } - forwarder, err := forward2.NewInterStepDataForward(u.VertexInstance.Vertex, reader, writers, conditionalForwarder, udfHandler, fetchWatermark, publishWatermark, opts...) + forwarder, err := forward.NewInterStepDataForward(u.VertexInstance.Vertex, reader, writers, conditionalForwarder, udfHandler, fetchWatermark, publishWatermark, opts...) if err != nil { return err } diff --git a/pkg/watermark/store/jetstream/kv_store.go b/pkg/watermark/store/jetstream/kv_store.go index b0e9c45aa4..001ed880a0 100644 --- a/pkg/watermark/store/jetstream/kv_store.go +++ b/pkg/watermark/store/jetstream/kv_store.go @@ -22,6 +22,8 @@ package jetstream import ( "context" "fmt" + "sync" + "time" "github.com/nats-io/nats.go" "go.uber.org/zap" @@ -36,6 +38,7 @@ type jetStreamStore struct { pipelineName string conn *jsclient.NatsConn kv nats.KeyValue + kvLock sync.RWMutex js *jsclient.JetStreamContext log *zap.SugaredLogger } @@ -45,7 +48,23 @@ var _ store.WatermarkKVStorer = (*jetStreamStore)(nil) // NewKVJetStreamKVStore returns KVJetStreamStore. func NewKVJetStreamKVStore(ctx context.Context, pipelineName string, bucketName string, client jsclient.JetStreamClient, opts ...JSKVStoreOption) (store.WatermarkKVStorer, error) { var err error - conn, err := client.Connect(ctx) + var jsStore *jetStreamStore + conn, err := client.Connect(ctx, jsclient.ReconnectHandler(func(_ *jsclient.NatsConn) { + if jsStore != nil && jsStore.js != nil { + // re-bind to an existing KeyValue store + kv, err := jsStore.js.KeyValue(bucketName) + // keep looping because the watermark won't work without the store + for err != nil { + jsStore.log.Errorw("Failed to rebind to the JetStream KeyValue store ", zap.Error(err)) + kv, err = jsStore.js.KeyValue(bucketName) + time.Sleep(100 * time.Millisecond) + } + jsStore.log.Infow("Succeeded to rebind to JetStream KeyValue store") + jsStore.kvLock.Lock() + defer jsStore.kvLock.Unlock() + jsStore.kv = kv + } + })) if err != nil { return nil, fmt.Errorf("failed to get nats connection, %w", err) } @@ -53,13 +72,11 @@ func NewKVJetStreamKVStore(ctx context.Context, pipelineName string, bucketName // do we need to specify any opts? if yes, send it via options. js, err := conn.JetStream(nats.PublishAsyncMaxPending(256)) if err != nil { - if !conn.IsClosed() { - conn.Close() - } + conn.Close() return nil, fmt.Errorf("failed to get JetStream context for writer") } - j := &jetStreamStore{ + jsStore = &jetStreamStore{ pipelineName: pipelineName, conn: conn, js: js, @@ -67,31 +84,29 @@ func NewKVJetStreamKVStore(ctx context.Context, pipelineName string, bucketName } // for JetStream KeyValue store, the bucket should have been created in advance - j.kv, err = j.js.KeyValue(bucketName) + jsStore.kv, err = jsStore.js.KeyValue(bucketName) if err != nil { - if !conn.IsClosed() { - conn.Close() - } + jsStore.Close() return nil, err } // options if any for _, o := range opts { - if err := o(j); err != nil { - if !conn.IsClosed() { - conn.Close() - } + if err := o(jsStore); err != nil { + jsStore.Close() return nil, err } } - return j, nil + return jsStore, nil } -// JSKVStoreOption is to pass in Jetstream options. +// JSKVStoreOption is to pass in JetStream options. type JSKVStoreOption func(*jetStreamStore) error // GetAllKeys returns all the keys in the key-value store. -func (kv *jetStreamStore) GetAllKeys(_ context.Context) ([]string, error) { - keys, err := kv.kv.Keys() +func (jss *jetStreamStore) GetAllKeys(_ context.Context) ([]string, error) { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() + keys, err := jss.kv.Keys() if err != nil { return nil, err } @@ -99,8 +114,10 @@ func (kv *jetStreamStore) GetAllKeys(_ context.Context) ([]string, error) { } // GetValue returns the value for a given key. -func (kv *jetStreamStore) GetValue(_ context.Context, k string) ([]byte, error) { - kvEntry, err := kv.kv.Get(k) +func (jss *jetStreamStore) GetValue(_ context.Context, k string) ([]byte, error) { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() + kvEntry, err := jss.kv.Get(k) if err != nil { return []byte(""), err } @@ -110,26 +127,34 @@ func (kv *jetStreamStore) GetValue(_ context.Context, k string) ([]byte, error) } // GetStoreName returns the store name. -func (kv *jetStreamStore) GetStoreName() string { - return kv.kv.Bucket() +func (jss *jetStreamStore) GetStoreName() string { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() + return jss.kv.Bucket() } // DeleteKey deletes the key from the JS key-value store. -func (kv *jetStreamStore) DeleteKey(_ context.Context, k string) error { +func (jss *jetStreamStore) DeleteKey(_ context.Context, k string) error { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() // will return error if nats connection is closed - return kv.kv.Delete(k) + return jss.kv.Delete(k) } // PutKV puts an element to the JS key-value store. -func (kv *jetStreamStore) PutKV(_ context.Context, k string, v []byte) error { +func (jss *jetStreamStore) PutKV(_ context.Context, k string, v []byte) error { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() // will return error if nats connection is closed - _, err := kv.kv.Put(k, v) + _, err := jss.kv.Put(k, v) return err } -// Close closes the jetstream connection. -func (kv *jetStreamStore) Close() { - if !kv.conn.IsClosed() { - kv.conn.Close() +// Close closes the JetStream connection. +func (jss *jetStreamStore) Close() { + jss.kvLock.RLock() + defer jss.kvLock.RUnlock() + if !jss.conn.IsClosed() { + jss.conn.Close() } } diff --git a/pkg/watermark/store/jetstream/kv_watch.go b/pkg/watermark/store/jetstream/kv_watch.go index 21658215a3..337d689742 100644 --- a/pkg/watermark/store/jetstream/kv_watch.go +++ b/pkg/watermark/store/jetstream/kv_watch.go @@ -32,8 +32,8 @@ import ( // jetStreamWatch implements the watermark's KV store backed up by Jetstream. type jetStreamWatch struct { pipelineName string + kvBucketName string conn *jsclient.NatsConn - kv nats.KeyValue js *jsclient.JetStreamContext log *zap.SugaredLogger } @@ -58,27 +58,18 @@ func NewKVJetStreamKVWatch(ctx context.Context, pipelineName string, kvBucketNam j := &jetStreamWatch{ pipelineName: pipelineName, + kvBucketName: kvBucketName, conn: conn, js: js, log: logging.FromContext(ctx).With("pipeline", pipelineName).With("kvBucketName", kvBucketName), } - j.kv, err = j.js.KeyValue(kvBucketName) - if err != nil { - if !conn.IsClosed() { - conn.Close() - } - return nil, err - } - // At this point, kvWatcher of type nats.KeyWatcher is nil // options if any for _, o := range opts { if err := o(j); err != nil { - if !conn.IsClosed() { - conn.Close() - } + j.Close() return nil, err } } @@ -111,37 +102,39 @@ func (k kvEntry) Operation() store.KVWatchOp { } // Watch watches the key-value store (aka bucket). -func (k *jetStreamWatch) Watch(ctx context.Context) (<-chan store.WatermarkKVEntry, <-chan struct{}) { - kvWatcher, err := k.kv.WatchAll(nats.IncludeHistory()) - for err != nil { - k.log.Errorw("WatchAll failed", zap.String("watcher", k.GetKVName()), zap.Error(err)) - kvWatcher, err = k.kv.WatchAll(nats.IncludeHistory()) - time.Sleep(100 * time.Millisecond) - } - +func (jsw *jetStreamWatch) Watch(ctx context.Context) (<-chan store.WatermarkKVEntry, <-chan struct{}) { + var err error + kvWatcher := jsw.newWatcher() var updates = make(chan store.WatermarkKVEntry) var stopped = make(chan struct{}) go func() { for { select { case <-ctx.Done(): - k.log.Infow("stopping WatchAll", zap.String("watcher", k.GetKVName())) - // call jetstream watch stop + jsw.log.Infow("stopping WatchAll", zap.String("watcher", jsw.GetKVName())) + // call JetStream watcher stop err = kvWatcher.Stop() if err != nil { - k.log.Errorw("Failed to stop", zap.String("watcher", k.GetKVName()), zap.Error(err)) + jsw.log.Errorw("Failed to stop", zap.String("watcher", jsw.GetKVName()), zap.Error(err)) } else { - k.log.Infow("WatchAll successfully stopped", zap.String("watcher", k.GetKVName())) + jsw.log.Infow("WatchAll successfully stopped", zap.String("watcher", jsw.GetKVName())) } close(updates) close(stopped) return - case value := <-kvWatcher.Updates(): - // if channel is closed, nil could come in + case value, ok := <-kvWatcher.Updates(): + if !ok { + // there are no more values to receive and the channel is closed, but context is not done yet + // meaning: there could be an auto reconnection to JetStream while the service is still running + // therefore, recreate the kvWatcher using the new JetStream context + kvWatcher = jsw.newWatcher() + jsw.log.Infow("Succeeded to recreate the watcher") + } if value == nil { + // watcher initialization and subscription send nil value continue } - k.log.Debug(value.Key(), value.Value(), value.Operation()) + jsw.log.Debug(value.Key(), value.Value(), value.Operation()) switch value.Operation() { case nats.KeyValuePut: updates <- kvEntry{ @@ -164,16 +157,34 @@ func (k *jetStreamWatch) Watch(ctx context.Context) (<-chan store.WatermarkKVEnt return updates, stopped } +func (jsw *jetStreamWatch) newWatcher() nats.KeyWatcher { + kv, err := jsw.js.KeyValue(jsw.kvBucketName) + // keep looping because the watermark won't work without a watcher + for err != nil { + jsw.log.Errorw("Failed to bind to the JetStream KeyValue store", zap.String("kvBucketName", jsw.kvBucketName), zap.String("watcher", jsw.GetKVName()), zap.Error(err)) + kv, err = jsw.js.KeyValue(jsw.kvBucketName) + time.Sleep(100 * time.Millisecond) + } + kvWatcher, err := kv.WatchAll(nats.IncludeHistory()) + // keep looping because the watermark won't work without a watcher + for err != nil { + jsw.log.Errorw("WatchAll failed", zap.String("watcher", jsw.GetKVName()), zap.Error(err)) + kvWatcher, err = kv.WatchAll(nats.IncludeHistory()) + time.Sleep(100 * time.Millisecond) + } + return kvWatcher +} + // GetKVName returns the KV store (bucket) name. -func (k *jetStreamWatch) GetKVName() string { - return k.kv.Bucket() +func (jsw *jetStreamWatch) GetKVName() string { + return jsw.kvBucketName } // Close closes the connection. -func (k *jetStreamWatch) Close() { +func (jsw *jetStreamWatch) Close() { // need to cancel the `Watch` ctx before calling Close() // otherwise `kvWatcher.Stop()` will raise the nats connection is closed error - if !k.conn.IsClosed() { - k.conn.Close() + if !jsw.conn.IsClosed() { + jsw.conn.Close() } } From 8dac0c6a17c492bade4b3a71e7330de458b556bc Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Fri, 13 Jan 2023 22:32:57 +0530 Subject: [PATCH 04/13] chore:adding replica information to reduce metrics (#473) Signed-off-by: Yashash H L --- .../server/service/pipeline_metrics_query.go | 18 ++-- pkg/forward/forward.go | 36 +++---- pkg/forward/metrics.go | 32 +++---- pkg/metrics/metrics.go | 7 +- pkg/reduce/metrics.go | 8 +- pkg/reduce/pbq/metrics.go | 4 +- pkg/reduce/pbq/pbq.go | 26 +++-- pkg/reduce/pbq/pbq_test.go | 9 +- pkg/reduce/pbq/pbqmanager.go | 38 +++++--- pkg/reduce/pbq/pbqmanager_test.go | 16 ++-- pkg/reduce/pbq/store/wal/metrics.go | 23 +++-- pkg/reduce/pbq/store/wal/segment.go | 94 +++++++++---------- pkg/reduce/pbq/store/wal/stores.go | 34 +++---- pkg/reduce/pnf/metrics.go | 12 +-- pkg/reduce/pnf/processandforward.go | 45 +++++++-- pkg/reduce/pnf/processandforward_test.go | 22 ++--- pkg/reduce/readloop/metrics.go | 16 ++-- pkg/reduce/readloop/ordered.go | 41 +++++--- pkg/reduce/readloop/ordered_test.go | 10 +- pkg/reduce/readloop/readloop.go | 58 +++++++++--- pkg/reduce/readloop/readloop_test.go | 10 +- pkg/reduce/reduce.go | 25 ++++- pkg/reduce/reduce_test.go | 48 +++++----- pkg/sinks/blackhole/blackhole.go | 4 +- pkg/sinks/blackhole/metrics.go | 4 +- pkg/sinks/kafka/kafka.go | 8 +- pkg/sinks/kafka/metrics.go | 8 +- pkg/sinks/logger/log.go | 4 +- pkg/sinks/logger/metrics.go | 4 +- pkg/sources/generator/metrics.go | 6 +- pkg/sources/generator/tickgen.go | 6 +- pkg/sources/http/http.go | 5 +- pkg/sources/http/metrics.go | 4 +- pkg/sources/kafka/metrics.go | 8 +- pkg/sources/kafka/reader.go | 8 +- pkg/sources/nats/metrics.go | 4 +- pkg/sources/nats/nats.go | 4 +- pkg/udf/reduce_udf.go | 4 +- 38 files changed, 415 insertions(+), 298 deletions(-) diff --git a/pkg/daemon/server/service/pipeline_metrics_query.go b/pkg/daemon/server/service/pipeline_metrics_query.go index 5447498491..4904357250 100644 --- a/pkg/daemon/server/service/pipeline_metrics_query.go +++ b/pkg/daemon/server/service/pipeline_metrics_query.go @@ -32,7 +32,7 @@ import ( "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/apis/proto/daemon" "github.com/numaproj/numaflow/pkg/isbsvc" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/watermark/fetch" ) @@ -177,12 +177,12 @@ func (ps *pipelineMetadataQuery) GetVertexMetrics(ctx context.Context, req *daem } // Check if the resultant metrics list contains the processingRate, if it does look for the period label - if value, ok := result[metricspkg.VertexProcessingRate]; ok { - metrics := value.GetMetric() - for _, metric := range metrics { + if value, ok := result[metrics.VertexProcessingRate]; ok { + metricsList := value.GetMetric() + for _, metric := range metricsList { labels := metric.GetLabel() for _, label := range labels { - if label.GetName() == metricspkg.LabelPeriod { + if label.GetName() == metrics.LabelPeriod { lookback := label.GetValue() processingRates[lookback] = metric.Gauge.GetValue() } @@ -190,12 +190,12 @@ func (ps *pipelineMetadataQuery) GetVertexMetrics(ctx context.Context, req *daem } } - if value, ok := result[metricspkg.VertexPendingMessages]; ok { - metrics := value.GetMetric() - for _, metric := range metrics { + if value, ok := result[metrics.VertexPendingMessages]; ok { + metricsList := value.GetMetric() + for _, metric := range metricsList { labels := metric.GetLabel() for _, label := range labels { - if label.GetName() == metricspkg.LabelPeriod { + if label.GetName() == metrics.LabelPeriod { lookback := label.GetValue() pendings[lookback] = int64(metric.Gauge.GetValue()) } diff --git a/pkg/forward/forward.go b/pkg/forward/forward.go index 1126b96338..2bffef6a59 100644 --- a/pkg/forward/forward.go +++ b/pkg/forward/forward.go @@ -34,7 +34,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" udfapplier "github.com/numaproj/numaflow/pkg/udf/applier" @@ -190,9 +190,9 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { readMessages, err := isdf.fromBuffer.Read(ctx, isdf.opts.readBatchSize) if err != nil { isdf.opts.logger.Warnw("failed to read fromBuffer", zap.Error(err)) - readMessagesError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() + readMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() } - readMessagesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readMessages))) + readMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readMessages))) // process only if we have any read messages. There is a natural looping here if there is an internal error while // reading, and we are not able to proceed. @@ -206,7 +206,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // to all the elements in the batch. If we were to assign last element's watermark, we will wrongly mark on-time data as late. processorWM := isdf.fetchWatermark.GetWatermark(readMessages[0].ReadOffset) for _, m := range readMessages { - readBytesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(m.Payload))) + readBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(m.Payload))) m.Watermark = time.Time(processorWM) if isdf.opts.vertexType == dfv1.VertexTypeSource && processorWM.After(m.EventTime) { // Set late data at source level m.IsLate = true @@ -250,14 +250,14 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // context.Done() is closed. wg.Wait() isdf.opts.logger.Debugw("concurrent applyUDF completed", zap.Int("concurrency", isdf.opts.udfConcurrency), zap.Duration("took", time.Since(concurrentUDFProcessingStart))) - concurrentUDFProcessingTime.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Observe(float64(time.Since(concurrentUDFProcessingStart).Microseconds())) + concurrentUDFProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Observe(float64(time.Since(concurrentUDFProcessingStart).Microseconds())) // Now that we know the UDF processing is done, let's figure out which vertex to send the results to. // Update the toBuffer(s) with writeMessages. for _, m := range udfResults { // look for errors in udf processing, if we see even 1 error let's return. handling partial retrying is not worth ATM. if m.udfError != nil { - udfError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() + udfError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() isdf.opts.logger.Errorw("failed to applyUDF", zap.Error(err)) return } @@ -303,13 +303,13 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // implicit return for posterity :-) if err != nil { isdf.opts.logger.Errorw("failed to ack from buffer", zap.Error(err)) - ackMessageError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readOffsets))) + ackMessageError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readOffsets))) return } - ackMessagesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readOffsets))) + ackMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(readOffsets))) // ProcessingTimes of the entire forwardAChunk - forwardAChunkProcessingTime.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "from": isdf.fromBuffer.GetName(), "to": toBuffers}).Observe(float64(time.Since(start).Microseconds())) + forwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "from": isdf.fromBuffer.GetName(), "to": toBuffers}).Observe(float64(time.Since(start).Microseconds())) } // ackFromBuffer acknowledges an array of offsets back to fromBuffer and is a blocking call or until shutdown has been initiated. @@ -397,11 +397,11 @@ retry: needRetry = true // we retry only failed messages failedMessages = append(failedMessages, messages[idx]) - writeMessagesError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Inc() + writeMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Inc() // a shutdown can break the blocking loop caused due to InternalErr if ok, _ := isdf.IsShuttingDown(); ok { err := fmt.Errorf("writeToBuffer failed, Stop called while stuck on an internal error with failed messages:%d, %v", len(failedMessages), errs) - platformError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName}).Inc() + platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() return writeOffsets, err } } else { @@ -423,8 +423,8 @@ retry: } } - writeMessagesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Add(float64(len(messages))) - writeBytesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Add(totalBytes) + writeMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Add(float64(len(messages))) + writeBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": toBuffer.GetName()}).Add(totalBytes) return writeOffsets, nil } @@ -432,12 +432,12 @@ retry: func (isdf *InterStepDataForward) concurrentApplyUDF(ctx context.Context, readMessagePair <-chan *readWriteMessagePair) { for message := range readMessagePair { start := time.Now() - udfReadMessagesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() + udfReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Inc() writeMessages, err := isdf.applyUDF(ctx, message.readMessage) - udfWriteMessagesCount.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(writeMessages))) + udfWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Add(float64(len(writeMessages))) message.writeMessages = append(message.writeMessages, writeMessages...) message.udfError = err - udfProcessingTime.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Observe(float64(time.Since(start).Microseconds())) + udfProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, "buffer": isdf.fromBuffer.GetName()}).Observe(float64(time.Since(start).Microseconds())) } } @@ -456,7 +456,7 @@ func (isdf *InterStepDataForward) applyUDF(ctx context.Context, readMessage *isb // this does not mean we should prohibit this from a shutdown. if ok, _ := isdf.IsShuttingDown(); ok { isdf.opts.logger.Errorw("UDF.Apply, Stop called while stuck on an internal error", zap.Error(err)) - platformError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName}).Inc() + platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() return nil, err } continue @@ -482,7 +482,7 @@ func (isdf *InterStepDataForward) whereToStep(writeMessage *isb.Message, message // a shutdown can break the blocking loop caused due to InternalErr if ok, _ := isdf.IsShuttingDown(); ok { err := fmt.Errorf("whereToStep, Stop called while stuck on an internal error, %v", err) - platformError.With(map[string]string{metricspkg.LabelVertex: isdf.vertexName, metricspkg.LabelPipeline: isdf.pipelineName}).Inc() + platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() return err } return err diff --git a/pkg/forward/metrics.go b/pkg/forward/metrics.go index e8bde1e26b..47cc6259a2 100644 --- a/pkg/forward/metrics.go +++ b/pkg/forward/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // readMessagesCount is used to indicate the number of messages read @@ -28,70 +28,70 @@ var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "read_total", Help: "Total number of Messages Read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // readBytesCount is to indicate the number of bytes read var readBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "read_bytes_total", Help: "Total number of bytes read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // readMessagesError is used to indicate the number of errors messages read var readMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "read_error_total", Help: "Total number of Read Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // writeMessagesCount is used to indicate the number of messages written var writeMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "write_total", Help: "Total number of Messages Written", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // writeBytesCount is to indicate the number of bytes written var writeBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "write_bytes_total", Help: "Total number of bytes written", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // writeMessagesError is used to indicate the number of errors messages written var writeMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "write_error_total", Help: "Total number of Write Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // ackMessagesCount is used to indicate the number of messages acknowledged var ackMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "ack_total", Help: "Total number of Messages Acknowledged", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // ackMessageError is used to indicate the errors in the number of messages acknowledged var ackMessageError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "ack_error_total", Help: "Total number of Acknowledged Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // udfError is used to indicate the number of UDF errors var udfError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "udf_error_total", Help: "Total number of UDF Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // platformError is used to indicate the number of Internal/Platform errors var platformError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "platform_error_total", Help: "Total number of platform Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) // forwardAChunkProcessingTime is a histogram to Observe forwardAChunk Processing times as a whole var forwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -99,7 +99,7 @@ var forwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramO Name: "forward_chunk_processing_time", Help: "Processing times of the entire forward a chunk (100 microseconds to 20 minutes)", Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 60), -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "from", "to"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "from", "to"}) // udfProcessingTime is a histogram to Observe UDF Processing times as a whole var udfProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -107,7 +107,7 @@ var udfProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ Name: "udf_processing_time", Help: "Processing times of UDF (100 microseconds to 15 minutes)", Buckets: prometheus.ExponentialBucketsRange(100, 60000000*15, 60), -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // concurrentUDFProcessingTime is a histogram to Observe UDF Processing times as a whole var concurrentUDFProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -115,18 +115,18 @@ var concurrentUDFProcessingTime = promauto.NewHistogramVec(prometheus.HistogramO Name: "concurrent_udf_processing_time", Help: "Processing times of Concurrent UDF (100 microseconds to 20 minutes)", Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 60), -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // udfReadMessagesCount is used to indicate the number of messages read by UDF var udfReadMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "udf_read_total", Help: "Total number of Messages Read at UDF", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) // udfWriteMessagesCount is used to indicate the number of messages read by UDF var udfWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "forwarder", Name: "udf_write_total", Help: "Total number of Messages Written at UDF", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 6c0a63b012..1041b21e7f 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -38,9 +38,10 @@ import ( ) const ( - LabelPipeline = "pipeline" - LabelVertex = "vertex" - LabelPeriod = "period" + LabelPipeline = "pipeline" + LabelVertex = "vertex" + LabelPeriod = "period" + LabelVertexReplicaIndex = "replica" VertexProcessingRate = "vertex_processing_rate" VertexPendingMessages = "vertex_pending_messages" diff --git a/pkg/reduce/metrics.go b/pkg/reduce/metrics.go index f68c871bf2..0e62133e60 100644 --- a/pkg/reduce/metrics.go +++ b/pkg/reduce/metrics.go @@ -1,7 +1,7 @@ package reduce import ( - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" ) @@ -11,18 +11,18 @@ var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_reader", Name: "read_total", Help: "Total number of Messages Read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) // readBytesCount is to indicate the number of bytes read var readBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_reader", Name: "read_bytes_total", Help: "Total number of bytes read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) // readMessagesError is used to indicate the number of read errors var readMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_reader", Name: "read_error_total", Help: "Total number of Read Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) diff --git a/pkg/reduce/pbq/metrics.go b/pkg/reduce/pbq/metrics.go index 065dc40430..cc600d8203 100644 --- a/pkg/reduce/pbq/metrics.go +++ b/pkg/reduce/pbq/metrics.go @@ -11,11 +11,11 @@ var activePartitionCount = promauto.NewGaugeVec(prometheus.GaugeOpts{ Subsystem: "reduce_pbq", Name: "active_partition_count", Help: "Total number of active partitions", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // pbqChannelSize is used to indicate the len of the pbq channel var pbqChannelSize = promauto.NewGaugeVec(prometheus.GaugeOpts{ Subsystem: "reduce_pbq", Name: "channel_size", Help: "PBQ Channel size", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) diff --git a/pkg/reduce/pbq/pbq.go b/pkg/reduce/pbq/pbq.go index 7c0d68679c..451f51e29a 100644 --- a/pkg/reduce/pbq/pbq.go +++ b/pkg/reduce/pbq/pbq.go @@ -19,6 +19,7 @@ package pbq import ( "context" "errors" + "strconv" "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" @@ -34,15 +35,16 @@ var ErrCOB = errors.New("error while writing to pbq, pbq is closed") // PBQ Buffer queue which is backed with a persisted store, each partition // will have a PBQ associated with it type PBQ struct { - vertexName string - pipelineName string - store store.Store - output chan *isb.ReadMessage - cob bool // cob to avoid panic in case writes happen after close of book - PartitionID partition.ID - options *options - manager *Manager - log *zap.SugaredLogger + vertexName string + pipelineName string + vertexReplica int32 + store store.Store + output chan *isb.ReadMessage + cob bool // cob to avoid panic in case writes happen after close of book + PartitionID partition.ID + options *options + manager *Manager + log *zap.SugaredLogger } var _ ReadWriteCloser = (*PBQ)(nil) @@ -67,7 +69,11 @@ func (p *PBQ) Write(ctx context.Context, message *isb.ReadMessage) error { close(p.output) writeErr = ctx.Err() } - pbqChannelSize.With(map[string]string{metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName}).Set(float64(len(p.output))) + pbqChannelSize.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Set(float64(len(p.output))) return writeErr } diff --git a/pkg/reduce/pbq/pbq_test.go b/pkg/reduce/pbq/pbq_test.go index 28b8301566..7aa2f276c5 100644 --- a/pkg/reduce/pbq/pbq_test.go +++ b/pkg/reduce/pbq/pbq_test.go @@ -26,8 +26,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/reduce/pbq/store" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" - + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/stretchr/testify/assert" ) @@ -41,7 +40,7 @@ func TestPBQ_ReadWrite(t *testing.T) { ctx := context.Background() - qManager, _ := NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(storeSize)), WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) + qManager, _ := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) // write 10 isb messages to persisted store msgCount := 10 @@ -102,7 +101,7 @@ func Test_PBQReadWithCanceledContext(t *testing.T) { ctx := context.Background() - qManager, err = NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(storeSize)), WithChannelBufferSize(int64(bufferSize)), WithReadTimeout(1*time.Second)) + qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), WithChannelBufferSize(int64(bufferSize)), WithReadTimeout(1*time.Second)) assert.NoError(t, err) @@ -168,7 +167,7 @@ func TestPBQ_WriteWithStoreFull(t *testing.T) { var err error ctx := context.Background() - qManager, err = NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(storeSize)), WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) + qManager, err = NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(storeSize)), WithChannelBufferSize(int64(buffSize)), WithReadTimeout(1*time.Second)) assert.NoError(t, err) // write 101 isb messages to pbq, but the store size is 100, we should get store is full error diff --git a/pkg/reduce/pbq/pbqmanager.go b/pkg/reduce/pbq/pbqmanager.go index d71493cd8e..dc120d22b3 100644 --- a/pkg/reduce/pbq/pbqmanager.go +++ b/pkg/reduce/pbq/pbqmanager.go @@ -20,10 +20,11 @@ import ( "context" "fmt" "math" + "strconv" "sync" "time" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/reduce/pbq/store" @@ -38,6 +39,7 @@ import ( type Manager struct { vertexName string pipelineName string + vertexReplica int32 storeProvider store.StoreProvider pbqOptions *options pbqMap map[string]*PBQ @@ -50,7 +52,7 @@ type Manager struct { // NewManager returns new instance of manager // We don't intend this to be called by multiple routines. -func NewManager(ctx context.Context, vertexName string, pipelineName string, storeProvider store.StoreProvider, opts ...PBQOption) (*Manager, error) { +func NewManager(ctx context.Context, vertexName string, pipelineName string, vr int32, storeProvider store.StoreProvider, opts ...PBQOption) (*Manager, error) { pbqOpts := DefaultOptions() for _, opt := range opts { if opt != nil { @@ -63,6 +65,7 @@ func NewManager(ctx context.Context, vertexName string, pipelineName string, sto pbqManager := &Manager{ vertexName: vertexName, pipelineName: pipelineName, + vertexReplica: vr, storeProvider: storeProvider, pbqMap: make(map[string]*PBQ), pbqOptions: pbqOpts, @@ -81,15 +84,16 @@ func (m *Manager) CreateNewPBQ(ctx context.Context, partitionID partition.ID) (R // output channel is buffered to support bulk reads p := &PBQ{ - vertexName: m.vertexName, - pipelineName: m.pipelineName, - store: persistentStore, - output: make(chan *isb.ReadMessage, m.pbqOptions.channelBufferSize), - cob: false, - PartitionID: partitionID, - options: m.pbqOptions, - manager: m, - log: logging.FromContext(ctx).With("PBQ", partitionID), + vertexName: m.vertexName, + pipelineName: m.pipelineName, + vertexReplica: m.vertexReplica, + store: persistentStore, + output: make(chan *isb.ReadMessage, m.pbqOptions.channelBufferSize), + cob: false, + PartitionID: partitionID, + options: m.pbqOptions, + manager: m, + log: logging.FromContext(ctx).With("PBQ", partitionID), } m.register(partitionID, p) return p, nil @@ -199,7 +203,11 @@ func (m *Manager) register(partitionID partition.ID, p *PBQ) { if _, ok := m.pbqMap[partitionID.String()]; !ok { m.pbqMap[partitionID.String()] = p } - activePartitionCount.With(map[string]string{metricspkg.LabelVertex: m.vertexName, metricspkg.LabelPipeline: m.pipelineName}).Inc() + activePartitionCount.With(map[string]string{ + metrics.LabelVertex: m.vertexName, + metrics.LabelPipeline: m.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(m.vertexReplica)), + }).Inc() } @@ -209,7 +217,11 @@ func (m *Manager) deregister(partitionID partition.ID) error { m.Lock() defer m.Unlock() delete(m.pbqMap, partitionID.String()) - activePartitionCount.With(map[string]string{metricspkg.LabelVertex: m.vertexName, metricspkg.LabelPipeline: m.pipelineName}).Dec() + activePartitionCount.With(map[string]string{ + metrics.LabelVertex: m.vertexName, + metrics.LabelPipeline: m.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(m.vertexReplica)), + }).Dec() return m.storeProvider.DeleteStore(partitionID) } diff --git a/pkg/reduce/pbq/pbqmanager_test.go b/pkg/reduce/pbq/pbqmanager_test.go index 1779dce5cb..3c5acf5b92 100644 --- a/pkg/reduce/pbq/pbqmanager_test.go +++ b/pkg/reduce/pbq/pbqmanager_test.go @@ -27,7 +27,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/testutils" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/reduce/pbq/store/noop" ) @@ -37,7 +37,7 @@ func TestManager_ListPartitions(t *testing.T) { size := int64(100) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) @@ -75,7 +75,7 @@ func TestManager_GetPBQ(t *testing.T) { size := int64(100) var pb1, pb2 ReadWriteCloser ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) @@ -99,7 +99,7 @@ func TestPBQFlow(t *testing.T) { size := int64(100) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) testPartition := partition.ID{ @@ -159,7 +159,7 @@ func TestPBQFlow(t *testing.T) { func TestPBQFlowWithNoOpStore(t *testing.T) { ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", noop.NewNoopStores(), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, noop.NewNoopStores(), WithReadTimeout(1*time.Second), WithChannelBufferSize(10)) assert.NoError(t, err) testPartition := partition.ID{ @@ -217,7 +217,7 @@ func TestManager_Replay(t *testing.T) { size := int64(100) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(size)), + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(size)), WithReadTimeout(1*time.Second), WithChannelBufferSize(10), WithReadBatchSize(10)) assert.NoError(t, err) testPartition := partition.ID{ @@ -289,9 +289,9 @@ func TestManager_StartUp(t *testing.T) { return []partition.ID{pID1, pID2}, nil } - stores := memory2.NewMemoryStores(memory2.WithStoreSize(size), memory2.WithDiscoverer(dp)) + stores := memory.NewMemoryStores(memory.WithStoreSize(size), memory.WithDiscoverer(dp)) ctx := context.Background() - pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", stores, + pbqManager, err := NewManager(ctx, "reduce", "test-pipeline", 0, stores, WithReadTimeout(1*time.Second), WithChannelBufferSize(10), WithReadBatchSize(10)) assert.NoError(t, err) ids, err := pbqManager.GetExistingPartitions(ctx) diff --git a/pkg/reduce/pbq/store/wal/metrics.go b/pkg/reduce/pbq/store/wal/metrics.go index 31d2256edf..cc49d0f122 100644 --- a/pkg/reduce/pbq/store/wal/metrics.go +++ b/pkg/reduce/pbq/store/wal/metrics.go @@ -20,12 +20,11 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) const ( - labelVertexReplicaIndex = "replica" - labelErrorKind = "kind" + labelErrorKind = "kind" ) // TODO - Adjust metric bucket range after we get more map reduce use cases. @@ -34,53 +33,53 @@ var entriesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "pbq_wal", Name: "wal_entries_total", Help: "Total number of entries written", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var entriesBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "pbq_wal", Name: "wal_entries_bytes_total", Help: "Total number of bytes written to WAL", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline, labelVertexReplicaIndex}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) var filesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "pbq_wal", Name: "wal_files_total", Help: "Total number of wal files/partitions (including both active and closed)", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var activeFilesCount = promauto.NewGaugeVec(prometheus.GaugeOpts{ Subsystem: "pbq_wal", Name: "active_wal_files_total", Help: "Total number of active wal files/partitions", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var garbageCollectingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ Subsystem: "pbq_wal", Name: "wal_garbage_collecting_time", Help: "Garbage Collecting time of a pbq wal (100 to 5000 microseconds)", Buckets: prometheus.ExponentialBucketsRange(100, 5000, 5), -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var fileSyncWaitTime = promauto.NewSummaryVec(prometheus.SummaryOpts{ Subsystem: "pbq_wal", Name: "wal_file_sync_wait_time", Help: "File Sync wait time", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var entryWriteLatency = promauto.NewSummaryVec(prometheus.SummaryOpts{ Subsystem: "pbq_wal", Name: "wal_entry_write_latency", Help: "Entry write time to WAL", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var entryEncodeLatency = promauto.NewSummaryVec(prometheus.SummaryOpts{ Subsystem: "pbq_wal", Name: "wal_entry_encode_latency", Help: "Time taken to encode an Entry", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) var walErrors = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "pbq_wal", Name: "wal_errors", Help: "Errors encountered", -}, []string{metricspkg.LabelPipeline, metricspkg.LabelVertex, labelVertexReplicaIndex, labelErrorKind}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex, labelErrorKind}) diff --git a/pkg/reduce/pbq/store/wal/segment.go b/pkg/reduce/pbq/store/wal/segment.go index c68d83a3a1..ea15136c7a 100644 --- a/pkg/reduce/pbq/store/wal/segment.go +++ b/pkg/reduce/pbq/store/wal/segment.go @@ -28,7 +28,7 @@ import ( "time" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" ) @@ -79,10 +79,10 @@ func (w *WAL) writeHeader() (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "writeHeader", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "writeHeader", }).Inc() } }() @@ -126,10 +126,10 @@ func (w *WAL) encodeHeader(id *partition.ID) (buf *bytes.Buffer, err error) { defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "encodeHeader", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "encodeHeader", }).Inc() } }() @@ -156,10 +156,10 @@ func (w *WAL) encodeEntry(message *isb.ReadMessage) (buf *bytes.Buffer, err erro defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "encodeEntry", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "encodeEntry", }).Inc() } }() @@ -214,10 +214,10 @@ func (w *WAL) encodeEntryHeader(message *isb.ReadMessage, messageLen int64, chec err = binary.Write(buf, binary.LittleEndian, entryHeader) if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "encodeEntryHeader", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "encodeEntryHeader", }).Inc() return nil, err } @@ -230,10 +230,10 @@ func (w *WAL) encodeEntryBody(message *isb.ReadMessage) (*bytes.Buffer, error) { err := enc.Encode(message.Message) if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "encodeEntryBody", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "encodeEntryBody", }).Inc() return nil, fmt.Errorf("entry body encountered encode err: %w", err) } @@ -256,19 +256,19 @@ func (w *WAL) Write(message *isb.ReadMessage) (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "write", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "write", }).Inc() } }() encodeStart := time.Now() entry, err := w.encodeEntry(message) entryEncodeLatency.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Observe(float64(time.Since(encodeStart).Milliseconds())) if err != nil { return err @@ -277,9 +277,9 @@ func (w *WAL) Write(message *isb.ReadMessage) (err error) { writeStart := time.Now() wrote, err := w.fp.WriteAt(entry.Bytes(), w.wOffset) entryWriteLatency.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Observe(float64(time.Since(writeStart).Milliseconds())) if wrote != entry.Len() { return fmt.Errorf("expected to write %d, but wrote only %d, %w", entry.Len(), wrote, err) @@ -292,14 +292,14 @@ func (w *WAL) Write(message *isb.ReadMessage) (err error) { // Only increase the write offset when we successfully write for atomicity. w.wOffset += int64(wrote) entriesBytesCount.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Add(float64(wrote)) entriesCount.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Inc() currentTime := time.Now() @@ -309,9 +309,9 @@ func (w *WAL) Write(message *isb.ReadMessage) (err error) { fSyncStart := time.Now() err = w.fp.Sync() fileSyncWaitTime.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Observe(float64(time.Since(fSyncStart).Milliseconds())) w.numOfUnsyncedMsgs = 0 return err @@ -324,19 +324,19 @@ func (w *WAL) Close() (err error) { defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), - labelErrorKind: "close", + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + labelErrorKind: "close", }).Inc() } }() start := time.Now() err = w.fp.Sync() fileSyncWaitTime.With(map[string]string{ - metricspkg.LabelPipeline: w.walStores.pipelineName, - metricspkg.LabelVertex: w.walStores.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), + metrics.LabelPipeline: w.walStores.pipelineName, + metrics.LabelVertex: w.walStores.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(w.walStores.replicaIndex)), }).Observe(float64(time.Since(start).Milliseconds())) if err != nil { diff --git a/pkg/reduce/pbq/store/wal/stores.go b/pkg/reduce/pbq/store/wal/stores.go index a1ba0957ae..6fa4eeb98d 100644 --- a/pkg/reduce/pbq/store/wal/stores.go +++ b/pkg/reduce/pbq/store/wal/stores.go @@ -26,7 +26,7 @@ import ( "time" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/reduce/pbq/store" ) @@ -90,14 +90,14 @@ func (ws *walStores) openOrCreateWAL(id *partition.ID) (*WAL, error) { return nil, err } filesCount.With(map[string]string{ - metricspkg.LabelPipeline: ws.pipelineName, - metricspkg.LabelVertex: ws.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Inc() activeFilesCount.With(map[string]string{ - metricspkg.LabelPipeline: ws.pipelineName, - metricspkg.LabelVertex: ws.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Inc() wal = &WAL{ fp: fp, @@ -207,10 +207,10 @@ func (ws *walStores) DeleteStore(partitionID partition.ID) error { defer func() { if err != nil { walErrors.With(map[string]string{ - metricspkg.LabelPipeline: ws.pipelineName, - metricspkg.LabelVertex: ws.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), - labelErrorKind: "gc", + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + labelErrorKind: "gc", }).Inc() } }() @@ -228,14 +228,14 @@ func (ws *walStores) DeleteStore(partitionID partition.ID) error { if err == nil { garbageCollectingTime.With(map[string]string{ - metricspkg.LabelPipeline: ws.pipelineName, - metricspkg.LabelVertex: ws.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Observe(float64(time.Since(start).Microseconds())) activeFilesCount.With(map[string]string{ - metricspkg.LabelPipeline: ws.pipelineName, - metricspkg.LabelVertex: ws.vertexName, - labelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Dec() } return err diff --git a/pkg/reduce/pnf/metrics.go b/pkg/reduce/pnf/metrics.go index fe27c49c7d..0598ce7e5e 100644 --- a/pkg/reduce/pnf/metrics.go +++ b/pkg/reduce/pnf/metrics.go @@ -11,28 +11,28 @@ var writeMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_writer", Name: "write_total", Help: "Total number of Messages Written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) // writeBytesCount is to indicate the number of bytes written var writeBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_writer", Name: "write_bytes_total", Help: "Total number of bytes written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) // writeMessagesError is used to indicate the number of errors messages written var writeMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_writer", Name: "write_error_total", Help: "Total number of Write Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, "buffer"}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, "buffer"}) // platformError is used to indicate the number of Internal/Platform errors var platformError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "platform", Name: "error_total", Help: "Total number of platform Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // reduceProcessTime reduce task processing latency var reduceProcessTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -40,7 +40,7 @@ var reduceProcessTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ Name: "process_time", Help: "Reduce process time (1 to 1200000 milliseconds)", Buckets: prometheus.ExponentialBucketsRange(1, 1200000, 5), -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // reduceForwardTime is used to indicate the time it took to forward the result var reduceForwardTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -48,4 +48,4 @@ var reduceForwardTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ Name: "forward_time", Help: "Reduce forward time (1 to 100000 microseconds)", Buckets: prometheus.ExponentialBucketsRange(1, 100000, 5), -}, []string{metrics.LabelPipeline, metrics.LabelVertex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) diff --git a/pkg/reduce/pnf/processandforward.go b/pkg/reduce/pnf/processandforward.go index 0b9b708eb3..ba0d39b6de 100644 --- a/pkg/reduce/pnf/processandforward.go +++ b/pkg/reduce/pnf/processandforward.go @@ -24,11 +24,12 @@ import ( "context" "errors" "math" + "strconv" "sync" "time" "github.com/numaproj/numaflow/pkg/forward" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" @@ -50,6 +51,7 @@ import ( type ProcessAndForward struct { vertexName string pipelineName string + vertexReplica int32 PartitionID partition.ID UDF applier.ReduceApplier result []*isb.Message @@ -64,6 +66,7 @@ type ProcessAndForward struct { func NewProcessAndForward(ctx context.Context, vertexName string, pipelineName string, + vr int32, partitionID partition.ID, udf applier.ReduceApplier, pbqReader pbq.Reader, @@ -72,6 +75,7 @@ func NewProcessAndForward(ctx context.Context, return &ProcessAndForward{ vertexName: vertexName, pipelineName: pipelineName, + vertexReplica: vr, PartitionID: partitionID, UDF: udf, pbqReader: pbqReader, @@ -86,7 +90,12 @@ func NewProcessAndForward(ctx context.Context, func (p *ProcessAndForward) Process(ctx context.Context) error { var err error startTime := time.Now() - defer reduceProcessTime.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName}).Observe(float64(time.Since(startTime).Milliseconds())) + defer reduceProcessTime.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Observe(float64(time.Since(startTime).Milliseconds())) + // blocking call, only returns the result after it has read all the messages from pbq p.result, err = p.UDF.ApplyReduce(ctx, &p.PartitionID, p.pbqReader.ReadCh()) return err @@ -96,13 +105,22 @@ func (p *ProcessAndForward) Process(ctx context.Context) error { func (p *ProcessAndForward) Forward(ctx context.Context) error { // extract window end time from the partitionID, which will be used for watermark startTime := time.Now() - defer reduceForwardTime.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName}).Observe(float64(time.Since(startTime).Microseconds())) + defer reduceForwardTime.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Observe(float64(time.Since(startTime).Microseconds())) + processorWM := processor.Watermark(p.PartitionID.End) // decide which ISB to write to to, err := p.whereToDecider.WhereTo(p.PartitionID.Key) if err != nil { - platformError.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName}).Inc() + platformError.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Inc() return err } messagesToStep := p.whereToStep(to) @@ -209,14 +227,27 @@ func (p *ProcessAndForward) writeToBuffer(ctx context.Context, bufferID string, if len(failedMessages) > 0 { p.log.Warnw("Failed to write messages to isb inside pnf", zap.Errors("errors", writeErrs)) writeMessages = failedMessages - writeMessagesError.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName}).Add(float64(len(failedMessages))) + writeMessagesError.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + "buffer": bufferID}).Add(float64(len(failedMessages))) return false, nil } return true, nil }) - writeMessagesCount.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName, "buffer": bufferID}).Add(float64(len(resultMessages))) - writeBytesCount.With(map[string]string{metricspkg.LabelVertex: p.vertexName, metricspkg.LabelPipeline: p.pipelineName, "buffer": bufferID}).Add(totalBytes) + writeMessagesCount.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + "buffer": bufferID}).Add(float64(len(resultMessages))) + + writeBytesCount.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + "buffer": bufferID}).Add(totalBytes) return offsets, ctxClosedErr } diff --git a/pkg/reduce/pnf/processandforward_test.go b/pkg/reduce/pnf/processandforward_test.go index 754794c539..e042278365 100644 --- a/pkg/reduce/pnf/processandforward_test.go +++ b/pkg/reduce/pnf/processandforward_test.go @@ -25,9 +25,9 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" - pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/numaproj/numaflow/pkg/watermark/processor" @@ -88,14 +88,14 @@ func TestProcessAndForward_Process(t *testing.T) { Key: "partition-1", } var err error - var pbqManager *pbq2.Manager + var pbqManager *pbq.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(100)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(100)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create a pbq for a partition - var simplePbq pbq2.ReadWriteCloser + var simplePbq pbq.ReadWriteCloser simplePbq, err = pbqManager.CreateNewPBQ(ctx, testPartition) assert.NoError(t, err) @@ -135,7 +135,7 @@ func TestProcessAndForward_Process(t *testing.T) { assert.NoError(t, err) _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(make(map[string]isb.BufferWriter)) // create pf using key and reducer - pf := NewProcessAndForward(ctx, "reduce", "test-pipeline", testPartition, client, simplePbq, make(map[string]isb.BufferWriter), myForwardTest{}, publishWatermark) + pf := NewProcessAndForward(ctx, "reduce", "test-pipeline", 0, testPartition, client, simplePbq, make(map[string]isb.BufferWriter), myForwardTest{}, publishWatermark) err = pf.Process(ctx) assert.NoError(t, err) @@ -145,9 +145,9 @@ func TestProcessAndForward_Process(t *testing.T) { func TestProcessAndForward_Forward(t *testing.T) { ctx := context.Background() - var pbqManager *pbq2.Manager + var pbqManager *pbq.Manager - pbqManager, _ = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores()) + pbqManager, _ = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores()) test1Buffer1 := simplebuffer.NewInMemoryBuffer("buffer1", 10) test1Buffer2 := simplebuffer.NewInMemoryBuffer("buffer2", 10) @@ -245,7 +245,7 @@ func TestProcessAndForward_Forward(t *testing.T) { } } -func createProcessAndForward(ctx context.Context, key string, pbqManager *pbq2.Manager, toBuffers map[string]isb.BufferWriter) ProcessAndForward { +func createProcessAndForward(ctx context.Context, key string, pbqManager *pbq.Manager, toBuffers map[string]isb.BufferWriter) ProcessAndForward { testPartition := partition.ID{ Start: time.UnixMilli(60000), @@ -255,7 +255,7 @@ func createProcessAndForward(ctx context.Context, key string, pbqManager *pbq2.M // create a pbq for a partition pw := buildPublisherMap(toBuffers) - var simplePbq pbq2.Reader + var simplePbq pbq.Reader simplePbq, _ = pbqManager.CreateNewPBQ(ctx, testPartition) resultPayload, _ := json.Marshal(PayloadForTest{ diff --git a/pkg/reduce/readloop/metrics.go b/pkg/reduce/readloop/metrics.go index 71fd0f1ec4..ac2886da11 100644 --- a/pkg/reduce/readloop/metrics.go +++ b/pkg/reduce/readloop/metrics.go @@ -15,28 +15,28 @@ var droppedMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_readloop", Name: "dropped_total", Help: "Total number of Messages Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, LabelReason}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, LabelReason}) // pbqWriteErrorCount is used to indicate the number of errors while writing to pbq var pbqWriteErrorCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_pbq", Name: "write_error_total", Help: "Total number of PBQ Write Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // pbqWriteMessagesCount is used to indicate the number of messages written to pbq var pbqWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_pbq", Name: "write_total", Help: "Total number of Messages Written to PBQ", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // partitionsInFlight is used to indicate the partitions in flight var partitionsInFlight = promauto.NewGaugeVec(prometheus.GaugeOpts{ Subsystem: "reduce_readloop", Name: "partitions_inflight", Help: "Total number of partitions in flight", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // pbqWriteTime pbq write latency var pbqWriteTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ @@ -44,25 +44,25 @@ var pbqWriteTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ Name: "write_time", Help: "Entry write time (1 to 5000 microseconds)", Buckets: prometheus.ExponentialBucketsRange(1, 5000, 5), -}, []string{metrics.LabelPipeline, metrics.LabelVertex}) +}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) // ackMessagesCount is used to indicate the number of messages acknowledged var ackMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_reader", Name: "ack_total", Help: "Total number of Messages Acknowledged", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // ackMessageError is used to indicate the errors in the number of messages acknowledged var ackMessageError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_isb_reader", Name: "ack_error_total", Help: "Total number of Acknowledged Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) // udfError is used to indicate the number of UDF errors var udfError = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "reduce_udf", Name: "error_total", Help: "Total number of UDF Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) diff --git a/pkg/reduce/readloop/ordered.go b/pkg/reduce/readloop/ordered.go index 6a996440dc..c3d659b45c 100644 --- a/pkg/reduce/readloop/ordered.go +++ b/pkg/reduce/readloop/ordered.go @@ -19,6 +19,7 @@ package readloop import ( "container/list" "context" + "strconv" "sync" "time" @@ -49,8 +50,9 @@ type task struct { // orderedForwarder orders the forwarding of the result of the execution of the tasks, even though the tasks itself are // run concurrently in an out of ordered fashion. type orderedForwarder struct { - vertexName string - pipelineName string + vertexName string + pipelineName string + vertexReplica int32 sync.RWMutex taskDone chan struct{} taskQueue *list.List @@ -58,13 +60,14 @@ type orderedForwarder struct { } // newOrderedForwarder returns an orderedForwarder. -func newOrderedForwarder(ctx context.Context, vertexName string, pipelineName string) *orderedForwarder { +func newOrderedForwarder(ctx context.Context, vertexName string, pipelineName string, vr int32) *orderedForwarder { of := &orderedForwarder{ - vertexName: vertexName, - pipelineName: pipelineName, - taskDone: make(chan struct{}), - taskQueue: list.New(), - log: logging.FromContext(ctx), + vertexName: vertexName, + pipelineName: pipelineName, + vertexReplica: vr, + taskDone: make(chan struct{}), + taskQueue: list.New(), + log: logging.FromContext(ctx), } go of.forward(ctx) @@ -87,13 +90,18 @@ func (of *orderedForwarder) schedulePnF(ctx context.Context, whereToDecider forward.ToWhichStepDecider, pw map[string]publish.Publisher) *task { - pf := pnf.NewProcessAndForward(ctx, of.vertexName, of.pipelineName, partitionID, udf, pbq, toBuffers, whereToDecider, pw) + pf := pnf.NewProcessAndForward(ctx, of.vertexName, of.pipelineName, of.vertexReplica, partitionID, udf, pbq, toBuffers, whereToDecider, pw) doneCh := make(chan struct{}) t := &task{ doneCh: doneCh, pf: pf, } - partitionsInFlight.With(map[string]string{metrics.LabelVertex: of.vertexName, metrics.LabelPipeline: of.pipelineName}).Inc() + partitionsInFlight.With(map[string]string{ + metrics.LabelVertex: of.vertexName, + metrics.LabelPipeline: of.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(of.vertexReplica)), + }).Inc() + // invoke the reduce function go of.reduceOp(ctx, t) return t @@ -110,7 +118,11 @@ func (of *orderedForwarder) reduceOp(ctx context.Context, t *task) { if err == nil { break } else if err == ctx.Err() { - udfError.With(map[string]string{metrics.LabelVertex: of.vertexName, metrics.LabelPipeline: of.pipelineName}).Inc() + udfError.With(map[string]string{ + metrics.LabelVertex: of.vertexName, + metrics.LabelPipeline: of.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(of.vertexReplica)), + }).Inc() of.log.Infow("ReduceOp exiting", zap.String("partitionID", t.pf.PartitionID.String()), zap.Error(ctx.Err())) return } @@ -182,7 +194,12 @@ func (of *orderedForwarder) forward(ctx context.Context) { rm := currElement currElement = currElement.Next() of.taskQueue.Remove(rm) - partitionsInFlight.With(map[string]string{metrics.LabelVertex: of.vertexName, metrics.LabelPipeline: of.pipelineName}).Dec() + partitionsInFlight.With(map[string]string{ + metrics.LabelVertex: of.vertexName, + metrics.LabelPipeline: of.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(of.vertexReplica)), + }).Dec() + of.log.Debugw("Removing task post forward call", zap.String("partitionID", t.pf.PartitionID.String())) of.Unlock() case <-ctx.Done(): diff --git a/pkg/reduce/readloop/ordered_test.go b/pkg/reduce/readloop/ordered_test.go index 1a2b77eb44..cc9ed70424 100644 --- a/pkg/reduce/readloop/ordered_test.go +++ b/pkg/reduce/readloop/ordered_test.go @@ -24,9 +24,9 @@ import ( "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/isb/testutils" - pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/stretchr/testify/assert" @@ -104,11 +104,11 @@ func TestOrderedProcessing(t *testing.T) { for _, tt := range tests { t.Run(tt.name, func(t *testing.T) { // clean out the task queue before we start a run - op := newOrderedForwarder(ctx, "reduce", "test-pipeline") + op := newOrderedForwarder(ctx, "reduce", "test-pipeline", 0) // although this could be declared outside, since we are using common naming scheme for partitions, // things will go haywire. - pbqManager, _ := pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(100)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + pbqManager, _ := pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(100)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) cCtx, cancelFn := context.WithCancel(ctx) defer cancelFn() for _, _partition := range tt.partitions { diff --git a/pkg/reduce/readloop/readloop.go b/pkg/reduce/readloop/readloop.go index be3b1d6a31..dd2176c16a 100644 --- a/pkg/reduce/readloop/readloop.go +++ b/pkg/reduce/readloop/readloop.go @@ -30,6 +30,7 @@ package readloop import ( "context" "math" + "strconv" "sync" "time" @@ -38,7 +39,7 @@ import ( "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/metrics" - pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/isb" @@ -54,8 +55,9 @@ import ( type ReadLoop struct { vertexName string pipelineName string + vertexReplica int32 UDF applier.ReduceApplier - pbqManager *pbq2.Manager + pbqManager *pbq.Manager windower window.Windower op *orderedForwarder log *zap.SugaredLogger @@ -69,18 +71,20 @@ type ReadLoop struct { func NewReadLoop(ctx context.Context, vertexName string, pipelineName string, + vr int32, udf applier.ReduceApplier, - pbqManager *pbq2.Manager, + pbqManager *pbq.Manager, windowingStrategy window.Windower, toBuffers map[string]isb.BufferWriter, whereToDecider forward.ToWhichStepDecider, pw map[string]publish.Publisher, ) (*ReadLoop, error) { - op := newOrderedForwarder(ctx, vertexName, pipelineName) + op := newOrderedForwarder(ctx, vertexName, pipelineName, vr) rl := &ReadLoop{ vertexName: vertexName, pipelineName: pipelineName, + vertexReplica: vr, UDF: udf, windower: windowingStrategy, pbqManager: pbqManager, @@ -176,7 +180,11 @@ messagesLoop: writtenMessages = append(writtenMessages, message) // let's not continue processing this message, most likely the window has already been closed and the message // won't be processed anyways. - droppedMessagesCount.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName, LabelReason: "watermark_issue"}).Inc() + droppedMessagesCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + LabelReason: "watermark_issue"}).Inc() continue } @@ -211,7 +219,12 @@ messagesLoop: // error loop, and we have received ctx.Done() via SIGTERM. func (rl *ReadLoop) writeToPBQ(ctx context.Context, p partition.ID, m *isb.ReadMessage) error { startTime := time.Now() - defer pbqWriteTime.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName}).Observe(float64(time.Since(startTime).Milliseconds())) + defer pbqWriteTime.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + }).Observe(float64(time.Since(startTime).Milliseconds())) + var pbqWriteBackoff = wait.Backoff{ Steps: math.MaxInt, Duration: 1 * time.Second, @@ -225,7 +238,11 @@ func (rl *ReadLoop) writeToPBQ(ctx context.Context, p partition.ID, m *isb.ReadM rErr := q.Write(context.Background(), m) if rErr != nil { rl.log.Errorw("Failed to write message", zap.Any("msgOffSet", m.ReadOffset.String()), zap.String("partitionID", p.String()), zap.Error(rErr)) - pbqWriteErrorCount.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName}).Inc() + pbqWriteErrorCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + }).Inc() // no point retrying if ctx.Done has been invoked select { case <-ctx.Done(): @@ -238,7 +255,11 @@ func (rl *ReadLoop) writeToPBQ(ctx context.Context, p partition.ID, m *isb.ReadM } // happy path - pbqWriteMessagesCount.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName}).Inc() + pbqWriteMessagesCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + }).Inc() rl.log.Debugw("Successfully wrote the message", zap.String("msgOffSet", m.ReadOffset.String()), zap.String("partitionID", p.String())) return true, nil }) @@ -267,7 +288,12 @@ func (rl *ReadLoop) ackMessages(ctx context.Context, messages []*isb.ReadMessage rErr := o.AckIt() attempt += 1 if rErr != nil { - ackMessageError.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName}).Inc() + ackMessageError.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + }).Inc() + rl.log.Errorw("Failed to ack message, retrying", zap.String("msgOffSet", o.String()), zap.Error(rErr), zap.Int("attempt", attempt)) // no point retrying if ctx.Done has been invoked select { @@ -280,7 +306,11 @@ func (rl *ReadLoop) ackMessages(ctx context.Context, messages []*isb.ReadMessage } } rl.log.Debugw("Successfully acked message", zap.String("msgOffSet", o.String())) - ackMessagesCount.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName}).Inc() + ackMessagesCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + }).Inc() return true, nil }) // no point trying the rest of the message, most likely that will also fail @@ -296,7 +326,7 @@ func (rl *ReadLoop) ackMessages(ctx context.Context, messages []*isb.ReadMessage // associatePBQAndPnF associates a PBQ with the partition if a PBQ exists, else creates a new one and then associates // it to the partition. -func (rl *ReadLoop) associatePBQAndPnF(ctx context.Context, partitionID partition.ID) pbq2.ReadWriteCloser { +func (rl *ReadLoop) associatePBQAndPnF(ctx context.Context, partitionID partition.ID) pbq.ReadWriteCloser { // look for existing pbq q := rl.pbqManager.GetPBQ(partitionID) @@ -341,7 +371,11 @@ func (rl *ReadLoop) upsertWindowsAndKeys(m *isb.ReadMessage) []window.AlignedKey // drop the late messages if m.IsLate { rl.log.Warnw("Dropping the late message", zap.Time("eventTime", m.EventTime), zap.Time("watermark", m.Watermark)) - droppedMessagesCount.With(map[string]string{metrics.LabelVertex: rl.vertexName, metrics.LabelPipeline: rl.pipelineName, LabelReason: "late"}).Inc() + droppedMessagesCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + LabelReason: "late"}).Inc() return []window.AlignedKeyedWindower{} } diff --git a/pkg/reduce/readloop/readloop_test.go b/pkg/reduce/readloop/readloop_test.go index ea6f1ee881..08e7d29974 100644 --- a/pkg/reduce/readloop/readloop_test.go +++ b/pkg/reduce/readloop/readloop_test.go @@ -11,9 +11,9 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" - pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/numaproj/numaflow/pkg/window/strategy/fixed" ) @@ -87,7 +87,7 @@ func TestReadLoop_Startup(t *testing.T) { }, } - memStoreProvider := memory2.NewMemoryStores(memory2.WithStoreSize(100)) + memStoreProvider := memory.NewMemoryStores(memory.WithStoreSize(100)) for _, id := range partitionIds { memStore, err := memStoreProvider.CreateStore(ctx, id) @@ -108,7 +108,7 @@ func TestReadLoop_Startup(t *testing.T) { } } - pManager, _ := pbq2.NewManager(ctx, "reduce", "test-pipeline", memStoreProvider, pbq2.WithChannelBufferSize(10)) + pManager, _ := pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memStoreProvider, pbq.WithChannelBufferSize(10)) to1 := simplebuffer.NewInMemoryBuffer("reduce-buffer", 4) toSteps := map[string]isb.BufferWriter{ @@ -119,7 +119,7 @@ func TestReadLoop_Startup(t *testing.T) { window := fixed.NewFixed(60 * time.Second) - rl, err := NewReadLoop(ctx, "reduce", "test-pipeline", &SumReduceTest{}, pManager, window, toSteps, &SumReduceTest{}, pw) + rl, err := NewReadLoop(ctx, "reduce", "test-pipeline", 0, &SumReduceTest{}, pManager, window, toSteps, &SumReduceTest{}, pw) assert.NoError(t, err) err = rl.Startup(ctx) assert.NoError(t, err) diff --git a/pkg/reduce/reduce.go b/pkg/reduce/reduce.go index efe52fad61..c51dba2cd1 100644 --- a/pkg/reduce/reduce.go +++ b/pkg/reduce/reduce.go @@ -21,6 +21,7 @@ package reduce import ( "context" + "strconv" "time" "github.com/numaproj/numaflow/pkg/forward" @@ -44,6 +45,7 @@ type DataForward struct { ctx context.Context vertexName string pipelineName string + vertexReplica int32 fromBuffer isb.BufferReader toBuffers map[string]isb.BufferWriter readloop *readloop.ReadLoop @@ -76,12 +78,13 @@ func NewDataForward(ctx context.Context, } rl, err := readloop.NewReadLoop(ctx, vertexInstance.Vertex.Spec.Name, vertexInstance.Vertex.Spec.PipelineName, - udf, pbqManager, windowingStrategy, toBuffers, whereToDecider, watermarkPublishers) + vertexInstance.Replica, udf, pbqManager, windowingStrategy, toBuffers, whereToDecider, watermarkPublishers) df := &DataForward{ ctx: ctx, vertexName: vertexInstance.Vertex.Spec.Name, pipelineName: vertexInstance.Vertex.Spec.PipelineName, + vertexReplica: vertexInstance.Replica, fromBuffer: fromBuffer, toBuffers: toBuffers, readloop: rl, @@ -150,13 +153,22 @@ func (d *DataForward) forwardAChunk(ctx context.Context) { totalBytes := 0 if err != nil { d.log.Errorw("Failed to read from isb", zap.Error(err)) - readMessagesError.With(map[string]string{metrics.LabelVertex: d.vertexName, metrics.LabelPipeline: d.pipelineName, "buffer": d.fromBuffer.GetName()}).Inc() + readMessagesError.With(map[string]string{ + metrics.LabelVertex: d.vertexName, + metrics.LabelPipeline: d.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(d.vertexReplica)), + "buffer": d.fromBuffer.GetName()}).Inc() } if len(readMessages) == 0 { return } - readMessagesCount.With(map[string]string{metrics.LabelVertex: d.vertexName, metrics.LabelPipeline: d.pipelineName, "buffer": d.fromBuffer.GetName()}).Add(float64(len(readMessages))) + readMessagesCount.With(map[string]string{ + metrics.LabelVertex: d.vertexName, + metrics.LabelPipeline: d.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(d.vertexReplica)), + "buffer": d.fromBuffer.GetName(), + }).Add(float64(len(readMessages))) // fetch watermark using the first element's watermark, because we assign the watermark to all other // elements in the batch based on the watermark we fetch from 0th offset. processorWM := d.watermarkFetcher.GetWatermark(readMessages[0].ReadOffset) @@ -168,6 +180,11 @@ func (d *DataForward) forwardAChunk(ctx context.Context) { m.Watermark = time.Time(processorWM) totalBytes += len(m.Payload) } - readBytesCount.With(map[string]string{metrics.LabelVertex: d.vertexName, metrics.LabelPipeline: d.pipelineName, "buffer": d.fromBuffer.GetName()}).Add(float64(totalBytes)) + readBytesCount.With(map[string]string{ + metrics.LabelVertex: d.vertexName, + metrics.LabelPipeline: d.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(d.vertexReplica)), + "buffer": d.fromBuffer.GetName(), + }).Add(float64(totalBytes)) d.readloop.Process(ctx, readMessages) } diff --git a/pkg/reduce/reduce_test.go b/pkg/reduce/reduce_test.go index eadd46570f..83ac2b26aa 100644 --- a/pkg/reduce/reduce_test.go +++ b/pkg/reduce/reduce_test.go @@ -31,9 +31,9 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" - pbq2 "github.com/numaproj/numaflow/pkg/reduce/pbq" + "github.com/numaproj/numaflow/pkg/reduce/pbq" "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" - memory2 "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" + "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/watermark/fetch" "github.com/numaproj/numaflow/pkg/watermark/processor" "github.com/numaproj/numaflow/pkg/watermark/publish" @@ -225,11 +225,11 @@ func TestDataForward_StartWithNoOpWM(t *testing.T) { } var err error - var pbqManager *pbq2.Manager + var pbqManager *pbq.Manager // create pbqManager - pbqManager, err = pbq2.NewManager(child, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(100)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + pbqManager, err = pbq.NewManager(child, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(100)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) publisher := map[string]publish.Publisher{ @@ -297,9 +297,9 @@ func TestReduceDataForward_Count(t *testing.T) { } // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(1000)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(1000)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher @@ -370,9 +370,9 @@ func TestReduceDataForward_Sum(t *testing.T) { } // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(1000)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(1000)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher @@ -443,9 +443,9 @@ func TestReduceDataForward_Max(t *testing.T) { } // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(1000)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(1000)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher @@ -516,9 +516,9 @@ func TestReduceDataForward_SumWithDifferentKeys(t *testing.T) { } // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(1000)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(1000)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher @@ -597,9 +597,9 @@ func TestReduceDataForward_NonKeyed(t *testing.T) { } // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(ctx, "reduce", "test-pipeline", memory2.NewMemoryStores(memory2.WithStoreSize(1000)), - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(ctx, "reduce", "test-pipeline", 0, memory.NewMemoryStores(memory.WithStoreSize(1000)), + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher @@ -674,12 +674,12 @@ func TestDataForward_WithContextClose(t *testing.T) { } // create a store provider - storeProvider := memory2.NewMemoryStores(memory2.WithStoreSize(1000)) + storeProvider := memory.NewMemoryStores(memory.WithStoreSize(1000)) // create pbq manager - var pbqManager *pbq2.Manager - pbqManager, err = pbq2.NewManager(cctx, "reduce", "test-pipeline", storeProvider, - pbq2.WithReadTimeout(1*time.Second), pbq2.WithChannelBufferSize(10)) + var pbqManager *pbq.Manager + pbqManager, err = pbq.NewManager(cctx, "reduce", "test-pipeline", 0, storeProvider, + pbq.WithReadTimeout(1*time.Second), pbq.WithChannelBufferSize(10)) assert.NoError(t, err) // create in memory watermark publisher and fetcher diff --git a/pkg/sinks/blackhole/blackhole.go b/pkg/sinks/blackhole/blackhole.go index cc5536382c..a24541da8c 100644 --- a/pkg/sinks/blackhole/blackhole.go +++ b/pkg/sinks/blackhole/blackhole.go @@ -30,7 +30,7 @@ import ( "go.uber.org/zap" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // Blackhole is a sink to emulate /dev/null @@ -95,7 +95,7 @@ func (b *Blackhole) IsFull() bool { // Write writes to the blackhole. func (b *Blackhole) Write(_ context.Context, messages []isb.Message) ([]isb.Offset, []error) { - sinkWriteCount.With(map[string]string{metricspkg.LabelVertex: b.name, metricspkg.LabelPipeline: b.pipelineName}).Add(float64(len(messages))) + sinkWriteCount.With(map[string]string{metrics.LabelVertex: b.name, metrics.LabelPipeline: b.pipelineName}).Add(float64(len(messages))) return nil, make([]error, len(messages)) } diff --git a/pkg/sinks/blackhole/metrics.go b/pkg/sinks/blackhole/metrics.go index 40aad0787d..0afcbb28b9 100644 --- a/pkg/sinks/blackhole/metrics.go +++ b/pkg/sinks/blackhole/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // sinkWriteCount is used to indicate the number of messages written to the sink @@ -28,4 +28,4 @@ var sinkWriteCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "blackhole_sink", Name: "write_total", Help: "Total number of messages written to blackhole sink", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sinks/kafka/kafka.go b/pkg/sinks/kafka/kafka.go index c6fdf75d82..2f9f0b25a7 100644 --- a/pkg/sinks/kafka/kafka.go +++ b/pkg/sinks/kafka/kafka.go @@ -31,7 +31,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/shared/util" ) @@ -165,7 +165,7 @@ func (tk *ToKafka) Write(_ context.Context, messages []isb.Message) ([]isb.Offse // Need to close and recreate later because the successes and errors channels might be unclean _ = tk.producer.Close() tk.connected = false - kafkaSinkWriteTimeouts.With(map[string]string{metricspkg.LabelVertex: tk.name, metricspkg.LabelPipeline: tk.pipelineName}).Inc() + kafkaSinkWriteTimeouts.With(map[string]string{metrics.LabelVertex: tk.name, metrics.LabelPipeline: tk.pipelineName}).Inc() close(done) return default: @@ -183,9 +183,9 @@ func (tk *ToKafka) Write(_ context.Context, messages []isb.Message) ([]isb.Offse <-done for _, err := range errs { if err != nil { - kafkaSinkWriteErrors.With(map[string]string{metricspkg.LabelVertex: tk.name, metricspkg.LabelPipeline: tk.pipelineName}).Inc() + kafkaSinkWriteErrors.With(map[string]string{metrics.LabelVertex: tk.name, metrics.LabelPipeline: tk.pipelineName}).Inc() } else { - kafkaSinkWriteCount.With(map[string]string{metricspkg.LabelVertex: tk.name, metricspkg.LabelPipeline: tk.pipelineName}).Inc() + kafkaSinkWriteCount.With(map[string]string{metrics.LabelVertex: tk.name, metrics.LabelPipeline: tk.pipelineName}).Inc() } } return nil, errs diff --git a/pkg/sinks/kafka/metrics.go b/pkg/sinks/kafka/metrics.go index 59c5244359..4770931940 100644 --- a/pkg/sinks/kafka/metrics.go +++ b/pkg/sinks/kafka/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // kafkaSinkWriteErrors is used to indicate the number of errors while while writing to kafka sink @@ -28,17 +28,17 @@ var kafkaSinkWriteErrors = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_sink", Name: "write_error_total", Help: "Total number of Write Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) // kafkaSinkWriteCount is used to indicate the number of messages written to kafka var kafkaSinkWriteCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_sink", Name: "write_total", Help: "Total number of errors on NewToKafka", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) var kafkaSinkWriteTimeouts = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_sink", Name: "write_timeout_total", Help: "Total number of write timeouts on NewToKafka", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sinks/logger/log.go b/pkg/sinks/logger/log.go index e1cf6c0731..763a46d355 100644 --- a/pkg/sinks/logger/log.go +++ b/pkg/sinks/logger/log.go @@ -30,7 +30,7 @@ import ( "go.uber.org/zap" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" ) @@ -98,7 +98,7 @@ func (s *ToLog) IsFull() bool { func (s *ToLog) Write(_ context.Context, messages []isb.Message) ([]isb.Offset, []error) { prefix := "(" + s.GetName() + ")" for _, message := range messages { - logSinkWriteCount.With(map[string]string{metricspkg.LabelVertex: s.name, metricspkg.LabelPipeline: s.pipelineName}).Inc() + logSinkWriteCount.With(map[string]string{metrics.LabelVertex: s.name, metrics.LabelPipeline: s.pipelineName}).Inc() log.Println(prefix, " Payload - ", string(message.Payload), " Key - ", message.Key, " Start - ", message.StartTime.UnixMilli(), " End - ", message.EndTime.UnixMilli()) } return nil, make([]error, len(messages)) diff --git a/pkg/sinks/logger/metrics.go b/pkg/sinks/logger/metrics.go index dcf6dd13b7..6a02c3a0e9 100644 --- a/pkg/sinks/logger/metrics.go +++ b/pkg/sinks/logger/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // logSinkWriteCount is used to indicate the number of messages written to log sink @@ -28,4 +28,4 @@ var logSinkWriteCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "log_sink", Name: "write_total", Help: "Total number of messages written to log sink", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sources/generator/metrics.go b/pkg/sources/generator/metrics.go index 468ab1e416..b9a4bbe0cc 100644 --- a/pkg/sources/generator/metrics.go +++ b/pkg/sources/generator/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // tickgenSourceReadCount is used to indicate the number of messages read by tick generator @@ -28,11 +28,11 @@ var tickgenSourceReadCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "tickgen_source", Name: "read_total", Help: "Total number of messages Read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) // tickgenSourceCount is used to indicate the number of times tickgen has ticked var tickgenSourceCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "tickgen_source", Name: "total", Help: "Total number of times tickgen source has ticked", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sources/generator/tickgen.go b/pkg/sources/generator/tickgen.go index a941b2ea98..50a7d4c098 100644 --- a/pkg/sources/generator/tickgen.go +++ b/pkg/sources/generator/tickgen.go @@ -32,7 +32,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/udf/applier" "github.com/numaproj/numaflow/pkg/watermark/fetch" @@ -225,7 +225,7 @@ loop: // we implement Read With Wait semantics select { case r := <-mg.srcchan: - tickgenSourceReadCount.With(map[string]string{metricspkg.LabelVertex: mg.name, metricspkg.LabelPipeline: mg.pipelineName}).Inc() + tickgenSourceReadCount.With(map[string]string{metrics.LabelVertex: mg.name, metrics.LabelPipeline: mg.pipelineName}).Inc() msgs = append(msgs, mg.newReadMessage(r.data, r.offset)) case <-timeout: mg.logger.Debugw("Timed out waiting for messages to read.", zap.Duration("waited", mg.readTimeout)) @@ -292,7 +292,7 @@ func (mg *memgen) generator(ctx context.Context, rate int, timeunit time.Duratio log.Info("Context.Done is called. exiting generator loop.") return case <-ticker.C: - tickgenSourceCount.With(map[string]string{metricspkg.LabelVertex: mg.name, metricspkg.LabelPipeline: mg.pipelineName}) + tickgenSourceCount.With(map[string]string{metrics.LabelVertex: mg.name, metrics.LabelPipeline: mg.pipelineName}) // swapped implies that the rcount is at limit if !atomic.CompareAndSwapInt32(&rcount, limit-1, limit) { go func() { diff --git a/pkg/sources/http/http.go b/pkg/sources/http/http.go index f2f0f42af6..5c977a915f 100644 --- a/pkg/sources/http/http.go +++ b/pkg/sources/http/http.go @@ -31,7 +31,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedtls "github.com/numaproj/numaflow/pkg/shared/tls" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" @@ -222,7 +222,8 @@ loop: oldest = m.EventTime } msgs = append(msgs, m) - httpSourceReadCount.With(map[string]string{metricspkg.LabelVertex: h.name, metricspkg.LabelPipeline: h.pipelineName}).Inc() + fmt.Println("got a message to read") + httpSourceReadCount.With(map[string]string{metrics.LabelVertex: h.name, metrics.LabelPipeline: h.pipelineName}).Inc() case <-timeout: h.logger.Debugw("Timed out waiting for messages to read.", zap.Duration("waited", h.readTimeout), zap.Int("read", len(msgs))) break loop diff --git a/pkg/sources/http/metrics.go b/pkg/sources/http/metrics.go index 2780d4fa3f..4512525a7f 100644 --- a/pkg/sources/http/metrics.go +++ b/pkg/sources/http/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // httpSourceReadCount is used to indicate the number of messages read by the http source vertex @@ -28,4 +28,4 @@ var httpSourceReadCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "http_source", Name: "read_total", Help: "Total number of messages Read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sources/kafka/metrics.go b/pkg/sources/kafka/metrics.go index f035498c0f..c966b28a2e 100644 --- a/pkg/sources/kafka/metrics.go +++ b/pkg/sources/kafka/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // kafkaSourceReadCount is used to indicate the number of messages read @@ -28,18 +28,18 @@ var kafkaSourceReadCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_source", Name: "read_total", Help: "Total number of messages Read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) // kafkaSourceAckCount is used to indicate the number of messages Acknowledged var kafkaSourceAckCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_source", Name: "ack_total", Help: "Total number of messages Acknowledged", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) // kafkaSourceOffsetAckErrors is used to indicate the number of errors while reading from kafka source with offsets var kafkaSourceOffsetAckErrors = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "kafka_source", Name: "ack_error_total", Help: "Total number of Kafka ID Errors", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sources/kafka/reader.go b/pkg/sources/kafka/reader.go index a6e9a1a4d3..4e591ad8c5 100644 --- a/pkg/sources/kafka/reader.go +++ b/pkg/sources/kafka/reader.go @@ -31,7 +31,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" "github.com/numaproj/numaflow/pkg/udf/applier" @@ -130,7 +130,7 @@ loop: for i := int64(0); i < count; i++ { select { case m := <-r.handler.messages: - kafkaSourceReadCount.With(map[string]string{metricspkg.LabelVertex: r.name, metricspkg.LabelPipeline: r.pipelineName}).Inc() + kafkaSourceReadCount.With(map[string]string{metrics.LabelVertex: r.name, metrics.LabelPipeline: r.pipelineName}).Inc() _m := toReadMessage(m) msgs = append(msgs, _m) // Get latest timestamps for different partitions @@ -173,14 +173,14 @@ func (r *KafkaSource) Ack(_ context.Context, offsets []isb.Offset) []error { for _, offset := range offsets { topic, partition, poffset, err := offsetFrom(offset.String()) if err != nil { - kafkaSourceOffsetAckErrors.With(map[string]string{metricspkg.LabelVertex: r.name, metricspkg.LabelPipeline: r.pipelineName}).Inc() + kafkaSourceOffsetAckErrors.With(map[string]string{metrics.LabelVertex: r.name, metrics.LabelPipeline: r.pipelineName}).Inc() r.logger.Errorw("Unable to extract partition offset of type int64 from the supplied offset. skipping and continuing", zap.String("suppliedoffset", offset.String()), zap.Error(err)) continue } // we need to mark the offset of the next message to read r.handler.sess.MarkOffset(topic, partition, poffset+1, "") - kafkaSourceAckCount.With(map[string]string{metricspkg.LabelVertex: r.name, metricspkg.LabelPipeline: r.pipelineName}).Inc() + kafkaSourceAckCount.With(map[string]string{metrics.LabelVertex: r.name, metrics.LabelPipeline: r.pipelineName}).Inc() } // How come it does not return errors at all? diff --git a/pkg/sources/nats/metrics.go b/pkg/sources/nats/metrics.go index 961b56e0cd..6369e457fa 100644 --- a/pkg/sources/nats/metrics.go +++ b/pkg/sources/nats/metrics.go @@ -20,7 +20,7 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" ) // natsSourceReadCount is used to indicate the number of messages read by the nats source vertex @@ -28,4 +28,4 @@ var natsSourceReadCount = promauto.NewCounterVec(prometheus.CounterOpts{ Subsystem: "nats_source", Name: "read_total", Help: "Total number of messages read", -}, []string{metricspkg.LabelVertex, metricspkg.LabelPipeline}) +}, []string{metrics.LabelVertex, metrics.LabelPipeline}) diff --git a/pkg/sources/nats/nats.go b/pkg/sources/nats/nats.go index 4fe059971b..13d831e529 100644 --- a/pkg/sources/nats/nats.go +++ b/pkg/sources/nats/nats.go @@ -28,7 +28,7 @@ import ( dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" - metricspkg "github.com/numaproj/numaflow/pkg/metrics" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedutil "github.com/numaproj/numaflow/pkg/shared/util" "github.com/numaproj/numaflow/pkg/udf/applier" @@ -220,7 +220,7 @@ loop: oldest = m.EventTime } msgs = append(msgs, m) - natsSourceReadCount.With(map[string]string{metricspkg.LabelVertex: ns.name, metricspkg.LabelPipeline: ns.pipelineName}).Inc() + natsSourceReadCount.With(map[string]string{metrics.LabelVertex: ns.name, metrics.LabelPipeline: ns.pipelineName}).Inc() case <-timeout: ns.logger.Debugw("Timed out waiting for messages to read.", zap.Duration("waited", ns.readTimeout), zap.Int("read", len(msgs))) break loop diff --git a/pkg/udf/reduce_udf.go b/pkg/udf/reduce_udf.go index 3f128fee1d..b60a6aeeda 100644 --- a/pkg/udf/reduce_udf.go +++ b/pkg/udf/reduce_udf.go @@ -136,7 +136,7 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { } // Readiness check if err := udfHandler.WaitUntilReady(ctx); err != nil { - return fmt.Errorf("failed on UDF readiness check, %w", err) + return fmt.Errorf("failed on FIXED_AGGREGATION readiness check, %w", err) } defer func() { err = udfHandler.CloseConn(ctx) @@ -148,7 +148,7 @@ func (u *ReduceUDFProcessor) Start(ctx context.Context) error { storeProvider := wal.NewWALStores(u.VertexInstance, wal.WithStorePath(dfv1.DefaultStorePath), wal.WithMaxBufferSize(dfv1.DefaultStoreMaxBufferSize), wal.WithSyncDuration(dfv1.DefaultStoreSyncDuration)) - pbqManager, err := pbq.NewManager(ctx, u.VertexInstance.Vertex.Spec.Name, u.VertexInstance.Vertex.Spec.PipelineName, storeProvider) + pbqManager, err := pbq.NewManager(ctx, u.VertexInstance.Vertex.Spec.Name, u.VertexInstance.Vertex.Spec.PipelineName, u.VertexInstance.Replica, storeProvider) if err != nil { log.Errorw("Failed to create pbq manager", zap.Error(err)) return fmt.Errorf("failed to create pbq manager, %w", err) From 9d133a8e89daf61ffc762a70ad083efba26744a4 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Fri, 13 Jan 2023 09:05:58 -0800 Subject: [PATCH 05/13] chore: finalize doc for 0.7 release (#472) Signed-off-by: Vigith Maurice --- README.md | 3 ++- docs/README.md | 3 ++- docs/quick-start.md | 8 +++++++- docs/user-guide/user-defined-functions/reduce/examples.md | 4 +++- docs/user-guide/user-defined-functions/reduce/reduce.md | 5 +++++ .../user-defined-functions/reduce/windowing/fixed.md | 4 +++- .../user-defined-functions/reduce/windowing/sliding.md | 6 ++++-- .../user-defined-functions/reduce/windowing/windowing.md | 2 ++ 8 files changed, 28 insertions(+), 7 deletions(-) diff --git a/README.md b/README.md index acdd5d4e7a..3e029b77a2 100644 --- a/README.md +++ b/README.md @@ -37,7 +37,8 @@ full-featured stream processing platforms. ## Roadmap -- Data aggregation (e.g. group-by) +- User Defined Transformer at Source for better deserialization and filter for cost reduction (v0.8) +- Multi partitioned edges for higher throughput (v0.9) ## Resources diff --git a/docs/README.md b/docs/README.md index 36190d9bfe..99ff0945a4 100644 --- a/docs/README.md +++ b/docs/README.md @@ -28,7 +28,8 @@ stream processing platforms. ## Roadmap -- Data aggregation (e.g. group-by) +- User Defined Transformer at Source for better deserialization and filter for cost reduction (v0.8) +- Multi partitioned edges for higher throughput (v0.9) ## Getting Started diff --git a/docs/quick-start.md b/docs/quick-start.md index 201e5b9b75..e7f396cba6 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -122,8 +122,14 @@ The pipeline can be deleted by kubectl delete -f https://raw.githubusercontent.com/numaproj/numaflow/stable/test/e2e/testdata/even-odd.yaml ``` +## A pipeline with reduce (aggregation) + +[Reduce Examples](user-guide/user-defined-functions/reduce/examples.md) + ## What's Next Try more examples in the [`examples`](https://github.com/numaproj/numaflow/tree/main/examples) directory. -After exploring how Numaflow pipeline run, you can check what data [Sources](./user-guide/sources/generator.md) and [Sinks](./user-guide/sinks/kafka.md) Numaflow supports out of the box, or learn how to write [User Defined Functions](user-guide/user-defined-functions/map/map.md). +After exploring how Numaflow pipeline run, you can check what data [Sources](./user-guide/sources/generator.md) +and [Sinks](./user-guide/sinks/kafka.md) Numaflow supports out of the box, or learn how to write +[User Defined Functions](user-guide/user-defined-functions/user-defined-functions.md). diff --git a/docs/user-guide/user-defined-functions/reduce/examples.md b/docs/user-guide/user-defined-functions/reduce/examples.md index 8cd86f2267..847b8f9063 100644 --- a/docs/user-guide/user-defined-functions/reduce/examples.md +++ b/docs/user-guide/user-defined-functions/reduce/examples.md @@ -1,5 +1,7 @@ # Reduce Examples +Please read [reduce](./reduce.md) to get the best out of these examples. + ## Prerequisites Install the ISB @@ -139,4 +141,4 @@ and 10, the output from the first reduce vertex will be 25 (5 * 5) and 50 (5 * 1 to the next non-keyed reduce vertex with the fixed window duration of 10s. This being a non-keyed, it will combine the inputs and produce the output of 150(25 * 2 + 50 * 2), which will be passed to the reduce vertex with a sliding window of duration 60s and with the slide duration of 10s. Hence the final output -will be 900(150 * 6). \ No newline at end of file +will be 900(150 * 6). diff --git a/docs/user-guide/user-defined-functions/reduce/reduce.md b/docs/user-guide/user-defined-functions/reduce/reduce.md index 1f4da8e42d..a8e6c6a2f9 100644 --- a/docs/user-guide/user-defined-functions/reduce/reduce.md +++ b/docs/user-guide/user-defined-functions/reduce/reduce.md @@ -1,5 +1,7 @@ # Reduce UDF +## Overview + Reduce is one of the most commonly used abstractions in a stream processing pipeline to define aggregation functions on a stream of data. It is the reduce feature that helps us solve problems like "performs a summary operation(such as counting the number of occurrence of a key, yielding user login @@ -9,6 +11,7 @@ bounding condition is "time", eg, "number of users logged in per minute". So whi unbounded stream of data, we need a way to group elements into finite chunks using time. To build these chunks the reduce function is applied to the set of records produced using the concept of [windowing](./windowing/windowing.md). +## Reduce Pseudo code Unlike in _map_ vertex where only an element is given to user-defined function, in _reduce_ since there is a group of elements, an iterator is passed to the reduce function. The following is a generic outlook of a reduce function. I have written the pseudo-code using the accumulator to show that very @@ -30,6 +33,8 @@ def reduceFn(key: str, datums: Iterator[Datum], md: Metadata) -> Messages: return Messages(Message.to_vtx(key, acumulator.result())) ``` +## Specification + The structure for defining a reduce vertex is as follows. ```yaml - name: my-reduce-udf diff --git a/docs/user-guide/user-defined-functions/reduce/windowing/fixed.md b/docs/user-guide/user-defined-functions/reduce/windowing/fixed.md index 45d9d062b8..ba748f3f4c 100644 --- a/docs/user-guide/user-defined-functions/reduce/windowing/fixed.md +++ b/docs/user-guide/user-defined-functions/reduce/windowing/fixed.md @@ -1,5 +1,7 @@ # Fixed +## Overview + Fixed windows (sometimes called tumbling windows) are defined by a static window size, e.g. 30 second windows, one minute windows, etc. They are generally aligned, i.e. every window applies across all the data for the corresponding period of time. It has a fixed size measured in time and does not @@ -24,7 +26,7 @@ vertices: NOTE: A duration string is a possibly signed sequence of decimal numbers, each with optional fraction and a unit suffix, such as "300ms", "1.5h" or "2h45m". Valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h". -## Length +### Length The `length` is the window size of the fixed window. diff --git a/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md b/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md index 3e407d4bca..f87995b111 100644 --- a/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md +++ b/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md @@ -1,5 +1,7 @@ # Sliding +## Overview + Sliding windows are similar to Fixed windows, the size of the windows is measured in time and is fixed. The important difference from the Fixed window is the fact that it allows an element to be present in more than one window. The additional window slide parameter controls how frequently a sliding window @@ -22,11 +24,11 @@ vertices: NOTE: A duration string is a possibly signed sequence of decimal numbers, each with optional fraction and a unit suffix, such as "300ms", "1.5h" or "2h45m". Valid time units are "ns", "us" (or "µs"), "ms", "s", "m", "h". -## Length +### Length The `length` is the window size of the fixed window. -## Slide +### Slide `slide` is the slide parameter that controls the frequency at which the sliding window is created. diff --git a/docs/user-guide/user-defined-functions/reduce/windowing/windowing.md b/docs/user-guide/user-defined-functions/reduce/windowing/windowing.md index 682ceae30f..c4cb8afe17 100644 --- a/docs/user-guide/user-defined-functions/reduce/windowing/windowing.md +++ b/docs/user-guide/user-defined-functions/reduce/windowing/windowing.md @@ -1,5 +1,7 @@ # Windowing +## Overview + In the world of data processing on an unbounded stream, Windowing is a concept of grouping data using temporal boundaries. We use event-time to discover temporal boundaries on an unbounded, infinite stream and [Watermark](../../../watermarks.md) to ensure From 766067022af4955b61dd52077a7573dcf47f5cf9 Mon Sep 17 00:00:00 2001 From: "github-actions[bot]" <41898282+github-actions[bot]@users.noreply.github.com> Date: Fri, 13 Jan 2023 09:22:43 -0800 Subject: [PATCH 06/13] docs: updated CHANGELOG.md (#474) Signed-off-by: GitHub --- CHANGELOG.md | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index a8c92a9af5..2b3cb09af8 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,28 @@ # Changelog +## v0.7.0 (2023-01-13) + + * [734e5d3](https://github.com/numaproj/numaflow/commit/734e5d3b44dee2ef690c9a1fe4d9d1ecb092a16c) Update manifests to v0.7.0 + * [5d6c533](https://github.com/numaproj/numaflow/commit/5d6c53369a6ea8da5f9f5036fce9aa81f6308fbf) fix: JetStream context KV store/watch fix (#460) + * [d6152e7](https://github.com/numaproj/numaflow/commit/d6152e772a03e646f2841a34497d498e4c2234c3) doc: reduce persistent store (#458) + * [ac77656](https://github.com/numaproj/numaflow/commit/ac77656dec8164fc162b8339bfff8138d17f73b0) doc: reduce documentation (#448) + * [257356a](https://github.com/numaproj/numaflow/commit/257356af0c932a9c7e84573c9f163e37a3e06dc4) chore(deps): bump json5 from 1.0.1 to 1.0.2 in /ui (#454) + * [7752db4](https://github.com/numaproj/numaflow/commit/7752db4b7d4233e2a691c0d1cc9ef2348dc75ab5) refactor: simplify http request construction in test cases (#444) + * [1a10af4](https://github.com/numaproj/numaflow/commit/1a10af4c20f051e46c063f9d946a39c208b6ec60) refactor: use exact matching instead of regex to perform e2e data validation. (#443) + * [2777e27](https://github.com/numaproj/numaflow/commit/2777e27ac0cdfcc954bf5e453b92b7f4e8a5c201) doc: windowing fixed and sliding (#439) + * [70fc008](https://github.com/numaproj/numaflow/commit/70fc008ffb0016a7310612d7cac191920207d0a6) refactor: move redis sink resources creation to E2ESuite (#437) + * [6c078b4](https://github.com/numaproj/numaflow/commit/6c078b42046b4733f702b3fbb585578d6304dafb) refactor: a prototype for enhancing E2E test framework (#424) + * [e7021c9](https://github.com/numaproj/numaflow/commit/e7021c9ae668724c11ac81fb49527ae8ce0f9240) feat: pipeline watermark (#416) + +### Contributors + + * Derek Wang + * Juanlu Yu + * Keran Yang + * Vedant Gupta + * Vigith Maurice + * dependabot[bot] + ## v0.7.0-rc1 (2022-12-16) * [71887db](https://github.com/numaproj/numaflow/commit/71887db5cce231b9b0a236f8f00ddeb0d40ac01a) Update manifests to v0.7.0-rc1 From 5ddf2046a499d740a21e5ca83099c0a83c4518e6 Mon Sep 17 00:00:00 2001 From: Derek Wang Date: Sun, 15 Jan 2023 23:26:56 -0800 Subject: [PATCH 07/13] chore: use alpine for dev image (#477) Signed-off-by: Derek Wang --- Dockerfile | 14 ++++++++------ Makefile | 12 ++++-------- docs/development/debugging.md | 6 ++++++ 3 files changed, 18 insertions(+), 14 deletions(-) diff --git a/Dockerfile b/Dockerfile index 938ad5e532..57787db67d 100644 --- a/Dockerfile +++ b/Dockerfile @@ -1,8 +1,9 @@ +ARG BASE_IMAGE=scratch ARG ARCH=$TARGETARCH #################################################################################################### # base #################################################################################################### -FROM alpine:3.12.3 as base +FROM alpine:3.17 as base ARG ARCH RUN apk update && apk upgrade && \ apk add ca-certificates && \ @@ -15,8 +16,8 @@ RUN chmod +x /bin/numaflow #################################################################################################### # numaflow #################################################################################################### -FROM scratch as numaflow -ARG ARCH +ARG BASE_IMAGE +FROM ${BASE_IMAGE} as numaflow COPY --from=base /usr/share/zoneinfo /usr/share/zoneinfo COPY --from=base /etc/ssl/certs/ca-certificates.crt /etc/ssl/certs/ca-certificates.crt COPY --from=base /bin/numaflow /bin/numaflow @@ -26,8 +27,7 @@ ENTRYPOINT [ "/bin/numaflow" ] #################################################################################################### # testbase #################################################################################################### -FROM alpine:3.12.3 as testbase -ARG ARCH +FROM alpine:3.17 as testbase RUN apk update && apk upgrade && \ apk add ca-certificates && \ apk --no-cache add tzdata @@ -35,7 +35,9 @@ RUN apk update && apk upgrade && \ COPY dist/e2eapi /bin/e2eapi RUN chmod +x /bin/e2eapi +#################################################################################################### +# testapi +#################################################################################################### FROM scratch AS e2eapi -ARG ARCH COPY --from=testbase /bin/e2eapi . ENTRYPOINT ["/e2eapi"] diff --git a/Makefile b/Makefile index 955c965139..04ff8730b9 100644 --- a/Makefile +++ b/Makefile @@ -2,6 +2,8 @@ CURRENT_DIR=$(shell pwd) DIST_DIR=${CURRENT_DIR}/dist BINARY_NAME:=numaflow DOCKERFILE:=Dockerfile +DEV_BASE_IMAGE:=alpine:3.17 +RELEASE_BASE_IMAGE:=scratch BUILD_DATE=$(shell date -u +'%Y-%m-%dT%H:%M:%SZ') GIT_COMMIT=$(shell git rev-parse HEAD) @@ -145,20 +147,14 @@ ui-test: ui-build .PHONY: image image: clean ui-build dist/$(BINARY_NAME)-linux-amd64 - DOCKER_BUILDKIT=1 $(DOCKER) build --build-arg "ARCH=amd64" -t $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) -f $(DOCKERFILE) . + DOCKER_BUILDKIT=1 $(DOCKER) build --build-arg "ARCH=amd64" --build-arg "BASE_IMAGE=$(DEV_BASE_IMAGE)" -t $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) -f $(DOCKERFILE) . @if [ "$(DOCKER_PUSH)" = "true" ]; then $(DOCKER) push $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION); fi ifdef IMAGE_IMPORT_CMD $(IMAGE_IMPORT_CMD) $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) endif -image-linux-%: dist/$(BINARY_NAME)-linux-$* - DOCKER_BUILDKIT=1 $(DOCKER) build --build-arg "ARCH=$*" -t $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION)-linux-$* --platform "linux/$*" --target $(BINARY_NAME) -f $(DOCKERFILE) . - @if [ "$(DOCKER_PUSH)" = "true" ]; then $(DOCKER) push $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION)-linux-$*; fi - - image-multi: ui-build set-qemu dist/$(BINARY_NAME)-linux-arm64.gz dist/$(BINARY_NAME)-linux-amd64.gz - $(DOCKER) buildx build --tag $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) --platform linux/amd64,linux/arm64 --file ./Dockerfile ${PUSH_OPTION} . - + $(DOCKER) buildx build --build-arg "BASE_IMAGE=$(RELEASE_BASE_IMAGE)" --tag $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) --platform linux/amd64,linux/arm64 --file ./Dockerfile ${PUSH_OPTION} . set-qemu: $(DOCKER) pull tonistiigi/binfmt:latest diff --git a/docs/development/debugging.md b/docs/development/debugging.md index 0f6e3d4e76..f1d7d77003 100644 --- a/docs/development/debugging.md +++ b/docs/development/debugging.md @@ -44,3 +44,9 @@ kubectl port-forward simple-pipeline-p1-0-7jzbn 2469 go tool pprof -http localhost:8081 https+insecure://localhost:2469/debug/pprof/heap ``` + +## Debug Inside the Container + +When doing local [development](development.md) using command lines such as `make start`, or `make image`, the built `numaflow` docker image is based on `alpine`, which allows you to execute into the container for debugging with `kubectl exec -it {pod-name} -c {container-name} -- sh`. + +This is not allowed when running pipelines with official released images, as they are based on `scratch`. From c2d3e9a8fbce0354f5cf3df2e549939d7bb95e65 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Mon, 16 Jan 2023 20:02:00 -0800 Subject: [PATCH 08/13] chore: set DefaultPBQChannelBufferSize to 100 from 10,000 (#484) Signed-off-by: Vigith Maurice --- pkg/apis/numaflow/v1alpha1/const.go | 2 +- pkg/window/keyed/keyed.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/pkg/apis/numaflow/v1alpha1/const.go b/pkg/apis/numaflow/v1alpha1/const.go index 3fefa8931a..08a68599c9 100644 --- a/pkg/apis/numaflow/v1alpha1/const.go +++ b/pkg/apis/numaflow/v1alpha1/const.go @@ -127,7 +127,7 @@ const ( DefaultReplicasPerScale = 2 // Default maximum replicas to be scaled up or down at once // Default persistent buffer queue options - DefaultPBQChannelBufferSize = 10000 // Default channel size in int + DefaultPBQChannelBufferSize = 100 // Default channel size in int (what should be right value?) DefaultPBQReadTimeout = 1 * time.Second // Default read timeout for pbq DefaultPBQReadBatchSize = 100 // Default read batch size for pbq diff --git a/pkg/window/keyed/keyed.go b/pkg/window/keyed/keyed.go index a8ed992d0c..734286cdef 100644 --- a/pkg/window/keyed/keyed.go +++ b/pkg/window/keyed/keyed.go @@ -93,6 +93,7 @@ func (kw *AlignedKeyedWindow) Keys() []string { idx := 0 for k := range kw.keys { keys[idx] = k + idx++ } return keys From 111f2b3b0796775a2ae51736bb9b572c50ccbb42 Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Wed, 18 Jan 2023 14:45:24 +0530 Subject: [PATCH 09/13] chore: fix sliding start strategy and minor optimizations (#486) Signed-off-by: Yashash H L --- .../reduce/windowing/sliding.md | 7 +- pkg/reduce/readloop/readloop.go | 4 +- pkg/window/strategy/sliding/sliding.go | 55 ++++++--------- pkg/window/strategy/sliding/sliding_test.go | 69 ++++++++++++++----- 4 files changed, 75 insertions(+), 60 deletions(-) diff --git a/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md b/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md index f87995b111..845dd2d1cb 100644 --- a/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md +++ b/docs/user-guide/user-defined-functions/reduce/windowing/sliding.md @@ -67,10 +67,9 @@ The window start time is always be left inclusive and right exclusive. That is w window is not considered active (it fell on the previous window, right exclusive) but `[2031-09-29T18:46:30Z, 2031-09-29T18:47:30Z)` is an active (left inclusive). -The first window always ends after _sliding_ seconds after the `time.now()` time. The start time itself will depend on the -time characteristics of the pipeline (event time, system time, etc.). This means the first window has its start time -in the past roughly given by `time.now() - duration(length) + duration(sliding)`. So the first window starts in the past -and ends _sliding_duration (based on time progression in the pipeline and not the wall time) from present. It is +The first window always ends after the sliding seconds from the `time.Now()`, the start time of the window will be the +nearest integer multiple of the slide which is less than the message's event time. So the first window starts in the +past and ends _sliding_duration (based on time progression in the pipeline and not the wall time) from present. It is important to note that regardless of the window boundary (starting in the past or ending in the future) the target element set totally depends on the matching time (in case of event time, all the elements with the time that falls with in the boundaries of the window, and in case of system time, all the elements that arrive from the `present` until the end of diff --git a/pkg/reduce/readloop/readloop.go b/pkg/reduce/readloop/readloop.go index dd2176c16a..a32f1f8c4a 100644 --- a/pkg/reduce/readloop/readloop.go +++ b/pkg/reduce/readloop/readloop.go @@ -384,9 +384,9 @@ func (rl *ReadLoop) upsertWindowsAndKeys(m *isb.ReadMessage) []window.AlignedKey for _, win := range processingWindows { w, isPresent := rl.windower.InsertIfNotPresent(win) if !isPresent { - rl.log.Debugw("Creating new keyed window", zap.Any("key", w.Keys()), zap.String("msg.offset", m.ID), zap.Int64("startTime", w.StartTime().UnixMilli()), zap.Int64("endTime", w.EndTime().UnixMilli())) + rl.log.Debugw("Creating new keyed window", zap.String("msg.offset", m.ID), zap.Int64("startTime", w.StartTime().UnixMilli()), zap.Int64("endTime", w.EndTime().UnixMilli())) } else { - rl.log.Debugw("Found an existing window", zap.Any("key", w.Keys()), zap.String("msg.offset", m.ID), zap.Int64("startTime", w.StartTime().UnixMilli()), zap.Int64("endTime", w.EndTime().UnixMilli())) + rl.log.Debugw("Found an existing window", zap.String("msg.offset", m.ID), zap.Int64("startTime", w.StartTime().UnixMilli()), zap.Int64("endTime", w.EndTime().UnixMilli())) } // track the key to window relationship w.AddKey(m.Key) diff --git a/pkg/window/strategy/sliding/sliding.go b/pkg/window/strategy/sliding/sliding.go index a5bdc5258e..ff4c76ecd0 100644 --- a/pkg/window/strategy/sliding/sliding.go +++ b/pkg/window/strategy/sliding/sliding.go @@ -65,47 +65,32 @@ func NewSliding(length time.Duration, slide time.Duration) *Sliding { // AssignWindow returns a set of windows that contain the element based on event time func (s *Sliding) AssignWindow(eventTime time.Time) []window.AlignedKeyedWindower { - // start time of the window in to which this element certainly belongs. - startTime := eventTime.Truncate(s.Length) - // end time of the window in to which this element certainly belongs. - endTime := startTime.Add(s.Length) - - // we have to find the boundaries of the sliding windows that are possible - // first lets consider end time as fixed, find the min start and end times - minEndTime := startTime.Add(s.Length % s.Slide) - minStartTime := minEndTime.Add(-s.Length) - - // lets consider start time as fixed and find the max start and end times. - maxStartTime := endTime.Add(-(s.Length % s.Slide)) - maxEndTime := maxStartTime.Add(s.Length) - - // now all the windows should fall in between maxend and minend times. - // one could consider min start and max start times as well. - wCount := int((maxEndTime.Sub(minEndTime)) / s.Slide) windows := make([]window.AlignedKeyedWindower, 0) - for i := 0; i < wCount; i++ { - // we make the windows left aligned since the original truncation operation - // is left aligned. - st := minStartTime.Add(time.Duration(i) * s.Slide) - et := st.Add(s.Length) - - // since there is overlap at the boundaries - // we attribute the element to the window to the right (higher) - // of the boundary - // left exclusive and right inclusive - // so given windows 500-600 and 600-700 and the event time is 600 - // we will add the element to 600-700 window and not to the 500-600 window. - if eventTime.Before(st) || !eventTime.Before(et) { - continue - } - - akw := keyed.NewKeyedWindow(st, et) + // use the highest integer multiple of slide length which is less than the eventTime + // as the start time for the window. For example if the eventTime is 810 and slide + // length is 70, use 770 as the startTime of the window. In that way we can be guarantee + // consistency while assigning the messages to the windows. + startTime := time.UnixMilli((eventTime.UnixMilli() / s.Slide.Milliseconds()) * s.Slide.Milliseconds()) + endTime := startTime.Add(s.Length) - windows = append(windows, akw) + // startTime and endTime will be the largest timestamp window for the given eventTime, + // using that we can create other windows by subtracting the slide length + + // since there is overlap at the boundaries + // we attribute the element to the window to the right (higher) + // of the boundary + // left exclusive and right inclusive + // so given windows 500-600 and 600-700 and the event time is 600 + // we will add the element to 600-700 window and not to the 500-600 window. + for !startTime.After(eventTime) && endTime.After(eventTime) { + windows = append(windows, keyed.NewKeyedWindow(startTime, endTime)) + startTime = startTime.Add(-s.Slide) + endTime = endTime.Add(-s.Slide) } return windows + } // InsertIfNotPresent inserts a window to the list of active windows if not present and returns the window diff --git a/pkg/window/strategy/sliding/sliding_test.go b/pkg/window/strategy/sliding/sliding_test.go index f039568470..5028791986 100644 --- a/pkg/window/strategy/sliding/sliding_test.go +++ b/pkg/window/strategy/sliding/sliding_test.go @@ -21,10 +21,9 @@ import ( "testing" "time" - "github.com/stretchr/testify/assert" - "github.com/numaproj/numaflow/pkg/window" "github.com/numaproj/numaflow/pkg/window/keyed" + "github.com/stretchr/testify/assert" ) // TestSliding_AssignWindow tests the assignment of element to a set of windows @@ -44,9 +43,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(10 * time.Second), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), - keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), }, }, { @@ -55,8 +54,8 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 40 * time.Second, eventTime: baseTime.Add(10 * time.Second), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), }, }, { @@ -65,8 +64,7 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 41 * time.Second, eventTime: baseTime.Add(10 * time.Second), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(559, 0), time.Unix(619, 0)), - keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(574, 0), time.Unix(634, 0)), }, }, { @@ -75,9 +73,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime, expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), - keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), }, }, { @@ -86,9 +84,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(time.Minute), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(620, 0), time.Unix(680, 0)), - keyed.NewKeyedWindow(time.Unix(640, 0), time.Unix(700, 0)), keyed.NewKeyedWindow(time.Unix(660, 0), time.Unix(720, 0)), + keyed.NewKeyedWindow(time.Unix(640, 0), time.Unix(700, 0)), + keyed.NewKeyedWindow(time.Unix(620, 0), time.Unix(680, 0)), }, }, { @@ -97,9 +95,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(time.Nanosecond), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), - keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), }, }, { @@ -108,9 +106,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(-time.Nanosecond), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(540, 0), time.Unix(600, 0)), - keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(620, 0)), + keyed.NewKeyedWindow(time.Unix(540, 0), time.Unix(600, 0)), }, }, { @@ -119,9 +117,9 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(20 * time.Second), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), - keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), keyed.NewKeyedWindow(time.Unix(620, 0), time.Unix(680, 0)), + keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), }, }, { @@ -130,9 +128,42 @@ func TestSliding_AssignWindow(t *testing.T) { slide: 20 * time.Second, eventTime: baseTime.Add(21 * time.Second), expected: []window.AlignedKeyedWindower{ - keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), - keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), keyed.NewKeyedWindow(time.Unix(620, 0), time.Unix(680, 0)), + keyed.NewKeyedWindow(time.Unix(600, 0), time.Unix(660, 0)), + keyed.NewKeyedWindow(time.Unix(580, 0), time.Unix(640, 0)), + }, + }, + { + name: "length not divisible by slide test 1", + length: time.Second * 600, + slide: 70 * time.Second, + eventTime: baseTime.Add(210 * time.Second), // 810 + expected: []window.AlignedKeyedWindower{ + keyed.NewKeyedWindow(time.Unix(770, 0), time.Unix(1370, 0)), + keyed.NewKeyedWindow(time.Unix(700, 0), time.Unix(1300, 0)), + keyed.NewKeyedWindow(time.Unix(630, 0), time.Unix(1230, 0)), + keyed.NewKeyedWindow(time.Unix(560, 0), time.Unix(1160, 0)), + keyed.NewKeyedWindow(time.Unix(490, 0), time.Unix(1090, 0)), + keyed.NewKeyedWindow(time.Unix(420, 0), time.Unix(1020, 0)), + keyed.NewKeyedWindow(time.Unix(350, 0), time.Unix(950, 0)), + keyed.NewKeyedWindow(time.Unix(280, 0), time.Unix(880, 0)), + }, + }, + { + name: "length not divisible by slide test 2", + length: time.Second * 600, + slide: 70 * time.Second, + eventTime: baseTime.Add(610 * time.Second), // 1210 + expected: []window.AlignedKeyedWindower{ + keyed.NewKeyedWindow(time.Unix(1190, 0), time.Unix(1790, 0)), + keyed.NewKeyedWindow(time.Unix(1120, 0), time.Unix(1720, 0)), + keyed.NewKeyedWindow(time.Unix(1050, 0), time.Unix(1650, 0)), + keyed.NewKeyedWindow(time.Unix(980, 0), time.Unix(1580, 0)), + keyed.NewKeyedWindow(time.Unix(910, 0), time.Unix(1510, 0)), + keyed.NewKeyedWindow(time.Unix(840, 0), time.Unix(1440, 0)), + keyed.NewKeyedWindow(time.Unix(770, 0), time.Unix(1370, 0)), + keyed.NewKeyedWindow(time.Unix(700, 0), time.Unix(1300, 0)), + keyed.NewKeyedWindow(time.Unix(630, 0), time.Unix(1230, 0)), }, }, } From a3024f4e26c5a3467c3b526f7083ab57948a8678 Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Thu, 19 Jan 2023 11:42:27 -0800 Subject: [PATCH 10/13] fix: -ve metrics and return early if isLate (#495) Signed-off-by: Vigith Maurice Signed-off-by: Yashash H L Co-authored-by: Yashash H L --- pkg/reduce/pbq/store/wal/stores.go | 21 ++++++++++++++------- pkg/reduce/readloop/readloop.go | 21 +++++++++++---------- 2 files changed, 25 insertions(+), 17 deletions(-) diff --git a/pkg/reduce/pbq/store/wal/stores.go b/pkg/reduce/pbq/store/wal/stores.go index 6fa4eeb98d..e90af94346 100644 --- a/pkg/reduce/pbq/store/wal/stores.go +++ b/pkg/reduce/pbq/store/wal/stores.go @@ -57,7 +57,7 @@ func NewWALStores(vertexInstance *dfv1.VertexInstance, opts ...Option) store.Sto return s } -func (ws *walStores) CreateStore(ctx context.Context, partitionID partition.ID) (store.Store, error) { +func (ws *walStores) CreateStore(_ context.Context, partitionID partition.ID) (store.Store, error) { // Create wal dir if not exist var err error if _, err = os.Stat(ws.storePath); os.IsNotExist(err) { @@ -77,6 +77,17 @@ func (ws *walStores) CreateStore(ctx context.Context, partitionID partition.ID) func (ws *walStores) openOrCreateWAL(id *partition.ID) (*WAL, error) { var err error + defer func() { + // increment active WAL count only if we are able to successfully create/open one. + if err == nil { + activeFilesCount.With(map[string]string{ + metrics.LabelPipeline: ws.pipelineName, + metrics.LabelVertex: ws.vertexName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), + }).Inc() + } + }() + filePath := getSegmentFilePath(id, ws.storePath) stat, err := os.Stat(filePath) @@ -89,16 +100,12 @@ func (ws *walStores) openOrCreateWAL(id *partition.ID) (*WAL, error) { if err != nil { return nil, err } + // we are interested only in the number of new files created filesCount.With(map[string]string{ metrics.LabelPipeline: ws.pipelineName, metrics.LabelVertex: ws.vertexName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), }).Inc() - activeFilesCount.With(map[string]string{ - metrics.LabelPipeline: ws.pipelineName, - metrics.LabelVertex: ws.vertexName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(ws.replicaIndex)), - }).Inc() wal = &WAL{ fp: fp, openMode: os.O_WRONLY, @@ -151,7 +158,7 @@ func (ws *walStores) openOrCreateWAL(id *partition.ID) (*WAL, error) { return wal, err } -func (ws *walStores) DiscoverPartitions(ctx context.Context) ([]partition.ID, error) { +func (ws *walStores) DiscoverPartitions(_ context.Context) ([]partition.ID, error) { files, err := os.ReadDir(ws.storePath) if os.IsNotExist(err) { return []partition.ID{}, nil diff --git a/pkg/reduce/readloop/readloop.go b/pkg/reduce/readloop/readloop.go index a32f1f8c4a..ff947b4f20 100644 --- a/pkg/reduce/readloop/readloop.go +++ b/pkg/reduce/readloop/readloop.go @@ -169,6 +169,17 @@ func (rl *ReadLoop) writeMessagesToWindows(ctx context.Context, messages []*isb. messagesLoop: for _, message := range messages { + // drop the late messages + if message.IsLate { + rl.log.Warnw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark)) + droppedMessagesCount.With(map[string]string{ + metrics.LabelVertex: rl.vertexName, + metrics.LabelPipeline: rl.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), + LabelReason: "late"}).Inc() + continue + } + // NOTE(potential bug): if we get a message where the event time is < watermark, skip processing the message. // This could be due to a couple of problem, eg. ack was not registered, etc. // Please do not confuse this with late data! This is a platform related problem causing the watermark inequality @@ -368,16 +379,6 @@ func (rl *ReadLoop) ShutDown(ctx context.Context) { // upsertWindowsAndKeys will create or assigns (if already present) a window to the message. It is an upsert operation // because windows are created out of order, but they will be closed in-order. func (rl *ReadLoop) upsertWindowsAndKeys(m *isb.ReadMessage) []window.AlignedKeyedWindower { - // drop the late messages - if m.IsLate { - rl.log.Warnw("Dropping the late message", zap.Time("eventTime", m.EventTime), zap.Time("watermark", m.Watermark)) - droppedMessagesCount.With(map[string]string{ - metrics.LabelVertex: rl.vertexName, - metrics.LabelPipeline: rl.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(rl.vertexReplica)), - LabelReason: "late"}).Inc() - return []window.AlignedKeyedWindower{} - } processingWindows := rl.windower.AssignWindow(m.EventTime) var kWindows []window.AlignedKeyedWindower From 2390e8a355f2846b9e1d3b81aa4bd983cc3cb092 Mon Sep 17 00:00:00 2001 From: Derek Wang Date: Thu, 19 Jan 2023 18:13:36 -0800 Subject: [PATCH 11/13] ci: disable docker build attestations (#497) Signed-off-by: Derek Wang --- Makefile | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/Makefile b/Makefile index 04ff8730b9..30d82ddd89 100644 --- a/Makefile +++ b/Makefile @@ -154,7 +154,7 @@ ifdef IMAGE_IMPORT_CMD endif image-multi: ui-build set-qemu dist/$(BINARY_NAME)-linux-arm64.gz dist/$(BINARY_NAME)-linux-amd64.gz - $(DOCKER) buildx build --build-arg "BASE_IMAGE=$(RELEASE_BASE_IMAGE)" --tag $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) --platform linux/amd64,linux/arm64 --file ./Dockerfile ${PUSH_OPTION} . + $(DOCKER) buildx build --sbom=false --provenance=false --build-arg "BASE_IMAGE=$(RELEASE_BASE_IMAGE)" --tag $(IMAGE_NAMESPACE)/$(BINARY_NAME):$(VERSION) --target $(BINARY_NAME) --platform linux/amd64,linux/arm64 --file ./Dockerfile ${PUSH_OPTION} . set-qemu: $(DOCKER) pull tonistiigi/binfmt:latest From 705cb76c060455230a439d6479d9dd83e131b83a Mon Sep 17 00:00:00 2001 From: Vigith Maurice Date: Thu, 19 Jan 2023 19:58:42 -0800 Subject: [PATCH 12/13] chore: enable pprof independently of debug flag (#496) Signed-off-by: Vigith Maurice --- docs/development/debugging.md | 3 ++- pkg/apis/numaflow/v1alpha1/const.go | 1 + pkg/metrics/metrics.go | 4 ++-- 3 files changed, 5 insertions(+), 3 deletions(-) diff --git a/docs/development/debugging.md b/docs/development/debugging.md index f1d7d77003..e035220fe4 100644 --- a/docs/development/debugging.md +++ b/docs/development/debugging.md @@ -34,7 +34,8 @@ spec: ## Profiling -Setting `NUMAFLOW_DEBUG` to `true` also enables `pprof` in the Vertex Pod. +If your pipeline is running with `NUMAFLOW_DEBUG` then `pprof` is enabled in the Vertex Pod. You +can also enable just `pprof` by setting `NUMAFLOW_PPROF` to `true`. For example, run the commands like below to profile memory usage for a Vertex Pod, a web page displaying the memory information will be automatically opened. diff --git a/pkg/apis/numaflow/v1alpha1/const.go b/pkg/apis/numaflow/v1alpha1/const.go index 08a68599c9..5495554b22 100644 --- a/pkg/apis/numaflow/v1alpha1/const.go +++ b/pkg/apis/numaflow/v1alpha1/const.go @@ -102,6 +102,7 @@ const ( EnvISBSvcJetStreamTLSEnabled = "NUMAFLOW_ISBSVC_JETSTREAM_TLS_ENABLED" EnvISBSvcConfig = "NUMAFLOW_ISBSVC_CONFIG" EnvDebug = "NUMAFLOW_DEBUG" + EnvPPROF = "NUMAFLOW_PPROF" PathVarRun = "/var/run/numaflow" VertexMetricsPort = 2469 diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 1041b21e7f..2793ace99d 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -252,8 +252,8 @@ func (ms *metricsServer) Start(ctx context.Context) (func(ctx context.Context) e } w.WriteHeader(http.StatusNoContent) }) - debugEnabled := os.Getenv(dfv1.EnvDebug) - if debugEnabled == "true" { + pprofEnabled := os.Getenv(dfv1.EnvDebug) == "true" || os.Getenv(dfv1.EnvPPROF) == "true" + if pprofEnabled { mux.HandleFunc("/debug/pprof/", pprof.Index) mux.HandleFunc("/debug/pprof/cmdline", pprof.Cmdline) mux.HandleFunc("/debug/pprof/profile", pprof.Profile) From afa5c760923909a108e2af320a5bd5a69fc622b2 Mon Sep 17 00:00:00 2001 From: Yashash H L Date: Sat, 21 Jan 2023 15:23:20 +0530 Subject: [PATCH 13/13] chore: optimise remove windows, adding test (#492) Signed-off-by: Yashash H L --- pkg/reduce/readloop/readloop.go | 29 ++-- pkg/window/strategy/fixed/fixed.go | 11 +- pkg/window/strategy/fixed/fixed_test.go | 21 +++ pkg/window/strategy/sliding/sliding.go | 14 +- .../sliding/sliding_benchmark_test.go | 133 ++++++++++++++++++ pkg/window/strategy/sliding/sliding_test.go | 21 +++ 6 files changed, 206 insertions(+), 23 deletions(-) create mode 100644 pkg/window/strategy/sliding/sliding_benchmark_test.go diff --git a/pkg/reduce/readloop/readloop.go b/pkg/reduce/readloop/readloop.go index ff947b4f20..09f2cda728 100644 --- a/pkg/reduce/readloop/readloop.go +++ b/pkg/reduce/readloop/readloop.go @@ -144,21 +144,26 @@ func (rl *ReadLoop) Process(ctx context.Context, messages []*isb.ReadMessage) { rl.log.Errorw("Failed to write messages", zap.Int("totalMessages", len(messages)), zap.Int("writtenMessage", len(successfullyWrittenMessages))) } + if len(successfullyWrittenMessages) == 0 { + return + } + // ack successful messages rl.ackMessages(ctx, successfullyWrittenMessages) + // close any windows that need to be closed. + // since the watermark will be same for all the messages in the batch + // we can invoke remove windows only once per batch + var closedWindows []window.AlignedKeyedWindower + wm := processor.Watermark(successfullyWrittenMessages[0].Watermark) - // for each successfully written message, - for _, m := range successfullyWrittenMessages { - // close any windows that need to be closed. - wm := processor.Watermark(m.Watermark) - closedWindows := rl.windower.RemoveWindows(time.Time(wm)) - rl.log.Debugw("Windows eligible for closing", zap.Int("length", len(closedWindows)), zap.Time("watermark", time.Time(wm))) - - for _, cw := range closedWindows { - partitions := cw.Partitions() - rl.closePartitions(partitions) - rl.log.Debugw("Closing Window", zap.Int64("windowStart", cw.StartTime().UnixMilli()), zap.Int64("windowEnd", cw.EndTime().UnixMilli())) - } + closedWindows = rl.windower.RemoveWindows(time.Time(wm)) + + rl.log.Debugw("Windows eligible for closing", zap.Int("length", len(closedWindows)), zap.Time("watermark", time.Time(wm))) + + for _, cw := range closedWindows { + partitions := cw.Partitions() + rl.closePartitions(partitions) + rl.log.Debugw("Closing Window", zap.Int64("windowStart", cw.StartTime().UnixMilli()), zap.Int64("windowEnd", cw.EndTime().UnixMilli())) } } diff --git a/pkg/window/strategy/fixed/fixed.go b/pkg/window/strategy/fixed/fixed.go index 19f16394a3..c258124b3e 100644 --- a/pkg/window/strategy/fixed/fixed.go +++ b/pkg/window/strategy/fixed/fixed.go @@ -143,11 +143,14 @@ func (f *Fixed) RemoveWindows(wm time.Time) []window.AlignedKeyedWindower { for e := f.entries.Front(); e != nil; { win := e.Value.(*keyed.AlignedKeyedWindow) next := e.Next() - // remove window only after the watermark has passed the end of the window - if win.EndTime().Before(wm) { - f.entries.Remove(e) - closedWindows = append(closedWindows, win) + + // break, if we find a window with end time > watermark + if !win.EndTime().Before(wm) { + break } + + f.entries.Remove(e) + closedWindows = append(closedWindows, win) e = next } diff --git a/pkg/window/strategy/fixed/fixed_test.go b/pkg/window/strategy/fixed/fixed_test.go index 719f20783b..583fe7504e 100644 --- a/pkg/window/strategy/fixed/fixed_test.go +++ b/pkg/window/strategy/fixed/fixed_test.go @@ -21,6 +21,7 @@ import ( "testing" "time" + "github.com/numaproj/numaflow/pkg/window" "github.com/stretchr/testify/assert" "github.com/numaproj/numaflow/pkg/window/keyed" @@ -357,6 +358,26 @@ func TestAligned_RemoveWindow(t *testing.T) { } } +func TestFixed_RemoveWindows(t *testing.T) { + var ( + length = time.Second * 60 + slidWin = NewFixed(length) + eventTime = time.Unix(60, 0) + expectedWindows = []window.AlignedKeyedWindower{ + keyed.NewKeyedWindow(time.Unix(60, 0), time.Unix(120, 0)), + keyed.NewKeyedWindow(time.Unix(120, 0), time.Unix(180, 0)), + keyed.NewKeyedWindow(time.Unix(180, 0), time.Unix(240, 0)), + } + ) + for i := 0; i < 10000; i++ { + win := keyed.NewKeyedWindow(eventTime, eventTime.Add(length)) + slidWin.InsertIfNotPresent(win) + eventTime = eventTime.Add(length) + } + closeWin := slidWin.RemoveWindows(time.Unix(300, 0)) + assert.Equal(t, closeWin, expectedWindows) +} + func setup(windows *Fixed, wins []*keyed.AlignedKeyedWindow) { windows.entries = list.New() for _, win := range wins { diff --git a/pkg/window/strategy/sliding/sliding.go b/pkg/window/strategy/sliding/sliding.go index ff4c76ecd0..cc48f49e10 100644 --- a/pkg/window/strategy/sliding/sliding.go +++ b/pkg/window/strategy/sliding/sliding.go @@ -144,9 +144,6 @@ func (s *Sliding) RemoveWindows(wm time.Time) []window.AlignedKeyedWindower { closedWindows := make([]window.AlignedKeyedWindower, 0) - if s.entries.Len() == 0 { - return closedWindows - } // examine the earliest window earliestWindow := s.entries.Front().Value.(*keyed.AlignedKeyedWindow) if earliestWindow.EndTime().After(wm) { @@ -157,11 +154,14 @@ func (s *Sliding) RemoveWindows(wm time.Time) []window.AlignedKeyedWindower { for e := s.entries.Front(); e != nil; { win := e.Value.(*keyed.AlignedKeyedWindow) next := e.Next() - // remove window only after the watermark has passed the end of the window - if win.EndTime().Before(wm) { - s.entries.Remove(e) - closedWindows = append(closedWindows, win) + + // break, if we find a window with end time > watermark + if !win.EndTime().Before(wm) { + break } + + s.entries.Remove(e) + closedWindows = append(closedWindows, win) e = next } diff --git a/pkg/window/strategy/sliding/sliding_benchmark_test.go b/pkg/window/strategy/sliding/sliding_benchmark_test.go new file mode 100644 index 0000000000..2af089a9e6 --- /dev/null +++ b/pkg/window/strategy/sliding/sliding_benchmark_test.go @@ -0,0 +1,133 @@ +package sliding + +import ( + "testing" + "time" + + "github.com/numaproj/numaflow/pkg/window/keyed" +) + +func BenchmarkSliding_AssignWindow(b *testing.B) { + AssignWindowHelper(b) + b.ReportAllocs() +} + +func BenchmarkSliding_InsertIfNotPresent(b *testing.B) { + InsertWindowHelper(b) + b.ReportAllocs() +} + +func BenchmarkSliding_RemoveWindowsWithCheck(b *testing.B) { + RemoveWindowWithCheckHelper(b) + b.ReportAllocs() +} + +func BenchmarkSliding_RemoveWindowsPerBatch(b *testing.B) { + RemoveWindowPerBatchHelper(b) + b.ReportAllocs() +} + +func InsertWindowHelper(b *testing.B) { + b.Helper() + var ( + //msgCount = 100 + eventTime = time.Unix(60, 0) + winLength = time.Second * 600 + sliding = time.Second * 60 + ) + windowStrat := NewSliding(winLength, sliding) + + for i := 0; i < b.N; i++ { + window := keyed.NewKeyedWindow(eventTime, eventTime.Add(winLength)) + windowStrat.InsertIfNotPresent(window) + eventTime = eventTime.Add(winLength) + } + +} + +func RemoveWindowWithCheckHelper(b *testing.B) { + b.Helper() + var ( + length = 10000 + eventTime = time.Unix(60, 0) + winLength = time.Second * 600 + sliding = time.Second * 60 + ) + windowStrat := NewSliding(winLength, sliding) + + b.StopTimer() + for i := 0; i < length; i++ { + window := keyed.NewKeyedWindow(eventTime, eventTime.Add(winLength)) + eventTime = eventTime.Add(winLength) + windowStrat.InsertIfNotPresent(window) + } + b.StartTimer() + + currentWatermark := time.Unix(60, 0) + batchSize := 500 + batchCount := 0 + latestWatermark := eventTime + for i := 0; i < b.N; i++ { + if currentWatermark.After(latestWatermark) { + _ = windowStrat.RemoveWindows(eventTime) + latestWatermark = eventTime + } + // update watermark once per batch + if batchCount == batchSize { + currentWatermark = currentWatermark.Add(winLength) + } + batchCount += 1 + } + +} + +func RemoveWindowPerBatchHelper(b *testing.B) { + b.Helper() + var ( + length = 10000 + eventTime = time.Unix(60, 0) + winLength = time.Second * 600 + sliding = time.Second * 60 + ) + windowStrat := NewSliding(winLength, sliding) + + b.StopTimer() + for i := 0; i < length; i++ { + window := keyed.NewKeyedWindow(eventTime, eventTime.Add(winLength)) + eventTime = eventTime.Add(winLength) + windowStrat.InsertIfNotPresent(window) + } + b.StartTimer() + + batchSize := 100 + batchCount := 0 + currentWatermark := time.Unix(60, 0) + for i := 0; i < b.N; i++ { + // invoke remove windows per batch + if batchCount == batchSize { + _ = windowStrat.RemoveWindows(currentWatermark) + } + // update watermark once in five batches + if batchCount == 5*batchSize { + currentWatermark = currentWatermark.Add(time.Second) + } + batchCount += 1 + } + +} + +func AssignWindowHelper(b *testing.B) { + b.Helper() + var ( + //msgCount = 100 + eventTime = time.Unix(60, 0) + winLength = time.Second * 600 + sliding = time.Second * 60 + ) + windowStrat := NewSliding(winLength, sliding) + + for i := 0; i < b.N; i++ { + windowStrat.AssignWindow(eventTime) + eventTime.Add(winLength) + } +} diff --git a/pkg/window/strategy/sliding/sliding_test.go b/pkg/window/strategy/sliding/sliding_test.go index 5028791986..92f5387cbe 100644 --- a/pkg/window/strategy/sliding/sliding_test.go +++ b/pkg/window/strategy/sliding/sliding_test.go @@ -297,6 +297,27 @@ func TestAligned_CreateWindow(t *testing.T) { } } +func TestSliding_RemoveWindows(t *testing.T) { + var ( + length = time.Second * 60 + slide = time.Second * 10 + slidWin = NewSliding(length, slide) + eventTime = time.Unix(60, 0) + expectedWindows = []window.AlignedKeyedWindower{ + keyed.NewKeyedWindow(time.Unix(60, 0), time.Unix(120, 0)), + keyed.NewKeyedWindow(time.Unix(120, 0), time.Unix(180, 0)), + keyed.NewKeyedWindow(time.Unix(180, 0), time.Unix(240, 0)), + } + ) + for i := 0; i < 10000; i++ { + win := keyed.NewKeyedWindow(eventTime, eventTime.Add(length)) + slidWin.InsertIfNotPresent(win) + eventTime = eventTime.Add(length) + } + closeWin := slidWin.RemoveWindows(time.Unix(300, 0)) + assert.Equal(t, closeWin, expectedWindows) +} + func setup(windows *Sliding, wins []*keyed.AlignedKeyedWindow) { windows.entries = list.New() for _, win := range wins {