diff --git a/config/base/dex/numaflow-dex-server-configmap.yaml b/config/base/dex/numaflow-dex-server-configmap.yaml index d7c02de5db..2569dcce8a 100644 --- a/config/base/dex/numaflow-dex-server-configmap.yaml +++ b/config/base/dex/numaflow-dex-server-configmap.yaml @@ -12,7 +12,7 @@ data: staticClients: - id: numaflow-server-app redirectURIs: - - //login + - //login name: 'Numaflow Server App' public: true connectors: diff --git a/config/install.yaml b/config/install.yaml index 8387d5efa1..beab7dacdf 100644 --- a/config/install.yaml +++ b/config/install.yaml @@ -16373,7 +16373,7 @@ metadata: apiVersion: v1 data: config.yaml: "issuer: /dex\nstorage:\n type: memory\nweb:\n http: 0.0.0.0:5556\nstaticClients:\n - \ - id: numaflow-server-app\n redirectURIs: \n - //login\n + \ - id: numaflow-server-app\n redirectURIs: \n - //login\n \ name: 'Numaflow Server App'\n public: true\nconnectors:\n- type: github\n \ # https://dexidp.io/docs/connectors/github/\n id: github\n name: GitHub\n \ config:\n clientID: $GITHUB_CLIENT_ID\n clientSecret: $GITHUB_CLIENT_SECRET\n diff --git a/config/namespace-install.yaml b/config/namespace-install.yaml index a34833a09a..98d494b5f6 100644 --- a/config/namespace-install.yaml +++ b/config/namespace-install.yaml @@ -16279,7 +16279,7 @@ metadata: apiVersion: v1 data: config.yaml: "issuer: /dex\nstorage:\n type: memory\nweb:\n http: 0.0.0.0:5556\nstaticClients:\n - \ - id: numaflow-server-app\n redirectURIs: \n - //login\n + \ - id: numaflow-server-app\n redirectURIs: \n - //login\n \ name: 'Numaflow Server App'\n public: true\nconnectors:\n- type: github\n \ # https://dexidp.io/docs/connectors/github/\n id: github\n name: GitHub\n \ config:\n clientID: $GITHUB_CLIENT_ID\n clientSecret: $GITHUB_CLIENT_SECRET\n diff --git a/docs/operations/metrics/metrics.md b/docs/operations/metrics/metrics.md index 03e7f1962f..f9b030ebee 100644 --- a/docs/operations/metrics/metrics.md +++ b/docs/operations/metrics/metrics.md @@ -12,113 +12,83 @@ These metrics can be used to determine throughput of your pipeline. #### Data-forward -| Metric name | Metric type | Labels | Description | -|---------------------------------------|-------------|------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------| -| `source_forwarder_read_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages read by a given Source Vertex | -| `source_forwarder_read_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes read by a given Source Vertex | -| `source_forwarder_write_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages written to Inter-Step Buffer by a given Source Vertex | -| `source_forwarder_write_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes written to Inter-Step Buffer by a given Source Vertex | -| `source_forwarder_ack_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages acknowledged by a given Source Vertex | -| `source_forwarder_drop_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages dropped by a given Source Vertex due to a full Inter-Step Buffer Partition | -| `source_forwarder_drop_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes dropped by a given Source Vertex due to a full Inter-Step Buffer Partition | -| `forwarder_data_read` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages read by a given Vertex from an Inter-Step Buffer Partition | -| `forwarder_read_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes read by a given Vertex from an Inter-Step Buffer Partition | -| `forwarder_write_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages written to Inter-Step Buffer by a given Vertex | -| `forwarder_write_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes written to Inter-Step Buffer by a given Vertex | -| `forwarder_ack_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages acknowledged by a given Vertex from an Inter-Step Buffer Partition | -| `forwarder_drop_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of messages dropped by a given Vertex due to a full Inter-Step Buffer Partition | -| `forwarder_drop_bytes_total` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Provides the total number of bytes dropped by a given Vertex due to a full Inter-Step Buffer Partition | -| `reduce_isb_reader_data_read` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of messages read by a given Reduce Vertex from an Inter-Step Buffer Partition | -| `reduce_isb_reader_read_bytes_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of bytes read by a given Reduce Vertex from an Inter-Step Buffer Partition | -| `reduce_isb_writer_write_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of messages written to Inter-Step Buffer by a given Reduce Vertex | -| `reduce_isb_writer_write_bytes_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of bytes written to Inter-Step Buffer by a given Reduce Vertex | -| `reduce_isb_writer_drop_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of messages dropped by a given Reduce Vertex due to a full Inter-Step Buffer Partition | -| `reduce_isb_writer_drop_bytes_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Provides the total number of bytes dropped by a given Reduce Vertex due to a full Inter-Step Buffer Partition | +| Metric name | Metric type | Labels | Description | +| ----------------------------- | ----------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------- | --------------------------------------------------------------------------------------------------------- | +| `forwarder_data_read_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of messages read by a given Vertex from an Inter-Step Buffer Partition | +| `forwarder_read_bytes_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of bytes read by a given Vertex from an Inter-Step Buffer Partition | +| `forwarder_write_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of messages written to Inter-Step Buffer by a given Vertex | +| `forwarder_write_bytes_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of bytes written to Inter-Step Buffer by a given Vertex | +| `forwarder_ack_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of messages acknowledged by a given Vertex from an Inter-Step Buffer Partition | +| `forwarder_drop_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of messages dropped by a given Vertex due to a full Inter-Step Buffer Partition | +| `forwarder_drop_bytes_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides the total number of bytes dropped by a given Vertex due to a full Inter-Step Buffer Partition | #### Kafka Source | Metric name | Metric type | Labels | Description | -|---------------------------|-------------|--------------------------------------------------------|-----------------------------------------------------------------------------------| -| `kafka_source_read_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages read by the Kafka Source Vertex/Processor. | -| `kafka_source_ack_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages acknowledged by the Kafka Source Vertex/Processor | - -#### Redis Streams Source - -| Metric name | Metric type | Labels | Description | -|---------------------------------------|-------------|--------------------------------------------------------|----------------------------------------------------------------------------------------------| -| `redis_streams_source_read_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages read by the Redis Streams Source Vertex/Processor | -| `redis_streams_source_read_err_total` | Counter | `vertex=`
`pipeline=` | Provides the number of read failures for the Redis Streams Source Vertex/Processor | -| `redis_streams_source_ack_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages acked by the Redis Streams Source Vertex/Processor | -| `redis_streams_source_ack_err_total` | Counter | `vertex=`
`pipeline=` | Provides the number of errors attempting to ack by the Redis Streams Source Vertex/Processor | +| ------------------------- | ----------- | ------------------------------------------------------ | --------------------------------------------------------------------------------- | +| `kafka_source_read_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages read by the Kafka Source Vertex/Processor. | +| `kafka_source_ack_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages acknowledged by the Kafka Source Vertex/Processor | #### Generator Source | Metric name | Metric type | Labels | Description | -|-----------------------------|-------------|--------------------------------------------------------|--------------------------------------------------------------------------------| -| `tickgen_source_read_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages read by the Generator Source Vertex/Processor. | +| --------------------------- | ----------- | ------------------------------------------------------ | ------------------------------------------------------------------------------ | +| `tickgen_source_read_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages read by the Generator Source Vertex/Processor. | #### Http Source | Metric name | Metric type | Labels | Description | -|--------------------------|-------------|--------------------------------------------------------|---------------------------------------------------------------------------| -| `http_source_read_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages read by the HTTP Source Vertex/Processor. | +| ------------------------ | ----------- | ------------------------------------------------------ | ------------------------------------------------------------------------- | +| `http_source_read_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages read by the HTTP Source Vertex/Processor. | #### Kafka Sink | Metric name | Metric type | Labels | Description | -|--------------------------|-------------|--------------------------------------------------------|----------------------------------------------------------------------------| -| `kafka_sink_write_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages written by the Kafka Sink Vertex/Processor | +| ------------------------ | ----------- | ------------------------------------------------------ | -------------------------------------------------------------------------- | +| `kafka_sink_write_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages written by the Kafka Sink Vertex/Processor | #### Log Sink | Metric name | Metric type | Labels | Description | -|------------------------|-------------|--------------------------------------------------------|--------------------------------------------------------------------------| -| `log_sink_write_total` | Counter | `vertex=`
`pipeline=` | Provides the number of messages written by the Log Sink Vertex/Processor | +| ---------------------- | ----------- | ------------------------------------------------------ | ------------------------------------------------------------------------ | +| `log_sink_write_total` | Counter | `pipeline=`
`vertex=` | Provides the number of messages written by the Log Sink Vertex/Processor | ### Latency These metrics can be used to determine the latency of your pipeline. -| Metric name | Metric type | Labels | Description | -|--------------------------------------------------|-------------|-----------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------| -| `source_forwarder_transformer_processing_time` | Histogram | `vertex=`
`pipeline=`
`partition_name=` | Provides a histogram distribution of the processing times of User Defined Source Transformer | -| `source_forwarder_forward_chunk_processing_time` | Histogram | `vertex=`
`pipeline=`
`partition_name=` | Provides a histogram distribution of the processing times of the source forwarder function as a whole | -| `forwarder_udf_processing_time` | Histogram | `vertex=`
`pipeline=`
`partition_name=` | Provides a histogram distribution of the processing times of User Defined Functions. (UDF's) | -| `forwarder_forward_chunk_processing_time` | Histogram | `vertex=`
`pipeline=`
`partition_name=` | Provides a histogram distribution of the processing times of the forwarder function as a whole | -| `reduce_pnf_process_time` | Histogram | `vertex=`
`pipeline=`
`replica=` | Provides a histogram distribution of the processing times of the reducer | -| `reduce_pnf_forward_time` | Histogram | `vertex=`
`pipeline=`
`replica=` | Provides a histogram distribution of the forwarding times of the reducer | +| Metric name | Metric type | Labels | Description | +| ---------------------------------------------- | ----------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------- | ---------------------------------------------------------------------------------------------- | +| `source_forwarder_transformer_processing_time` | Histogram | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Provides a histogram distribution of the processing times of User Defined Source Transformer | +| `forwarder_udf_processing_time` | Histogram | `pipeline=`
`vertex=`
`vertex_type=`
`replica=` | Provides a histogram distribution of the processing times of User Defined Functions. (UDF's) | +| `forwarder_forward_chunk_processing_time` | Histogram | `pipeline=`
`vertex=`
`vertex_type=`
`replica=` | Provides a histogram distribution of the processing times of the forwarder function as a whole | +| `reduce_pnf_process_time` | Histogram | `pipeline=`
`vertex=`
`replica=` | Provides a histogram distribution of the processing times of the reducer | +| `reduce_pnf_forward_time` | Histogram | `pipeline=`
`vertex=`
`replica=` | Provides a histogram distribution of the forwarding times of the reducer | ### Errors These metrics can be used to determine if there are any errors in the pipeline -| Metric name | Metric type | Labels | Description | -|---------------------------------------|-------------|------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------| -| `source_forwarder_platform_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any internal errors which could stop pipeline processing | -| `source_forwarder_read_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while reading messages by the source forwarder | -| `source_forwarder_write_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while writing messages by the source forwarder | -| `source_forwarder_ack_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while acknowledging messages by the source forwarder | -| `forwarder_platform_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any internal errors which could stop pipeline processing | -| `forwarder_read_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while reading messages by the forwarder | -| `forwarder_write_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while writing messages by the forwarder | -| `forwarder_ack_error` | Counter | `vertex=`
`pipeline=`
`partition_name=` | Indicates any errors while acknowledging messages by the forwarder | -| `kafka_source_offset_ack_errors` | Counter | `vertex=`
`pipeline=` | Indicates any kafka acknowledgement errors | -| `kafka_sink_write_error_total` | Counter | `vertex=`
`pipeline=` | Provides the number of errors while writing to the Kafka sink | -| `kafka_sink_write_timeout_total` | Counter | `vertex=`
`pipeline=` | Provides the write timeouts while writing to the Kafka sink | -| `isb_jetstream_read_error_total` | Counter | `partition_name=` | Indicates any read errors with NATS Jetstream ISB | -| `isb_jetstream_write_error_total` | Counter | `partition_name=` | Indicates any write errors with NATS Jetstream ISB | -| `isb_redis_read_error_total` | Counter | `partition_name=` | Indicates any read errors with Redis ISB | -| `isb_redis_write_error_total` | Counter | `partition_name=` | Indicates any write errors with Redis ISB | -| `reduce_isb_reader_read_error_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Indicates any read errors with Reducer ISB | -| `reduce_isb_writer_write_error_total` | Counter | `vertex=`
`pipeline=`
`replica=`
`partition_name=` | Indicates any write errors with Reducer ISB | -| `reduce_pnf_platform_error_total` | Counter | `vertex=`
`pipeline=`
`replica=` | Indicates any internal errors while processing and forwarding data by reducer | +| Metric name | Metric type | Labels | Description | +| --------------------------------- | ----------- | ------------------------------------------------------------------------------------------------------------------------------------------------------------- | ------------------------------------------------------------------ | +| `forwarder_platform_error_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=` | Indicates any internal errors which could stop pipeline processing | +| `forwarder_read_error_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Indicates any errors while reading messages by the forwarder | +| `forwarder_write_error_total` | Counter | `pipeline=`
`vertex=` `vertex_type=`

`replica=`
`partition_name=` | Indicates any errors while writing messages by the forwarder | +| `forwarder_ack_error_total` | Counter | `pipeline=`
`vertex=`
`vertex_type=`
`replica=`
`partition_name=` | Indicates any errors while acknowledging messages by the forwarder | +| `kafka_source_offset_ack_errors` | Counter | `pipeline=`
`vertex=` | Indicates any kafka acknowledgement errors | +| `kafka_sink_write_error_total` | Counter | `pipeline=`
`vertex=` | Provides the number of errors while writing to the Kafka sink | +| `kafka_sink_write_timeout_total` | Counter | `pipeline=`
`vertex=` | Provides the write timeouts while writing to the Kafka sink | +| `isb_jetstream_read_error_total` | Counter | `partition_name=` | Indicates any read errors with NATS Jetstream ISB | +| `isb_jetstream_write_error_total` | Counter | `partition_name=` | Indicates any write errors with NATS Jetstream ISB | +| `isb_redis_read_error_total` | Counter | `partition_name=` | Indicates any read errors with Redis ISB | +| `isb_redis_write_error_total` | Counter | `partition_name=` | Indicates any write errors with Redis ISB | ### Saturation #### NATS JetStream ISB | Metric name | Metric type | Labels | Description | -|------------------------------------|-------------|------------------------|----------------------------------------------------------------------------------------------------------------------------------------------| +| ---------------------------------- | ----------- | ---------------------- | -------------------------------------------------------------------------------------------------------------------------------------------- | | `isb_jetstream_isFull_total` | Counter | `buffer=` | Indicates if the ISB is full. Continual increase of this counter metric indicates a potential backpressure that can be built on the pipeline | | `isb_jetstream_buffer_soft_usage` | Gauge | `buffer=` | Indicates the usage/utilization of a NATS Jetstream ISB | | `isb_jetstream_buffer_solid_usage` | Gauge | `buffer=` | Indicates the solid usage of a NATS Jetstream ISB | @@ -128,7 +98,7 @@ These metrics can be used to determine if there are any errors in the pipeline #### Redis ISB | Metric name | Metric type | Labels | Description | -|--------------------------|-------------|------------------------|----------------------------------------------------------------------------------------------------------------------------------------------| +| ------------------------ | ----------- | ---------------------- | -------------------------------------------------------------------------------------------------------------------------------------------- | | `isb_redis_isFull_total` | Counter | `buffer=` | Indicates if the ISB is full. Continual increase of this counter metric indicates a potential backpressure that can be built on the pipeline | | `isb_redis_buffer_usage` | Gauge | `buffer=` | Indicates the usage/utilization of a Redis ISB | | `isb_redis_consumer_lag` | Gauge | `buffer=` | Indicates the the consumer lag of a Redis ISB | diff --git a/pkg/daemon/server/service/rater/helper.go b/pkg/daemon/server/service/rater/helper.go index 514d2c5a85..f680eb946c 100644 --- a/pkg/daemon/server/service/rater/helper.go +++ b/pkg/daemon/server/service/rater/helper.go @@ -24,17 +24,6 @@ import ( const ( indexNotFound = -1 - - // The following string set is used to identify the vertex type of pod - - // keyVertexTypeReduce is the vertex type string for reduce vertex - keyVertexTypeReduce = "reduce" - // keyVertexTypeSource is the vertex type string for a source vertex - keyVertexTypeSource = "source" - // keyVertexTypeSink is the vertex type string for a sink vertex - keyVertexTypeSink = "sink" - // keyVertexTypeOther is the vertex type string for other vertices - keyVertexTypeOther = "other" ) // UpdateCount updates the count of processed messages for a pod at a given time diff --git a/pkg/daemon/server/service/rater/pod_tracker.go b/pkg/daemon/server/service/rater/pod_tracker.go index 5876e3c404..4fb2fc49a5 100644 --- a/pkg/daemon/server/service/rater/pod_tracker.go +++ b/pkg/daemon/server/service/rater/pod_tracker.go @@ -98,10 +98,9 @@ func (pt *PodTracker) trackActivePods(ctx context.Context) { func (pt *PodTracker) updateActivePods() { for _, v := range pt.pipeline.Spec.Vertices { - vType := getVertexType(v) for i := 0; i < int(v.Scale.GetMaxReplicas()); i++ { podName := fmt.Sprintf("%s-%s-%d", pt.pipeline.Name, v.Name, i) - podKey := pt.getPodKey(i, v.Name, vType) + podKey := pt.getPodKey(i, v.Name) if pt.isActive(v.Name, podName) { pt.activePods.PushBack(podKey) } else { @@ -112,19 +111,6 @@ func (pt *PodTracker) updateActivePods() { pt.log.Debugf("Finished updating the active pod set: %v", pt.activePods.ToString()) } -func getVertexType(v v1alpha1.AbstractVertex) string { - switch { - case v.IsReduceUDF(): - return keyVertexTypeReduce - case v.IsASource(): - return keyVertexTypeSource - case v.IsASink(): - return keyVertexTypeSink - default: - return keyVertexTypeOther - } -} - // LeastRecentlyUsed returns the least recently used pod from the active pod list. // if there are no active pods, it returns an empty string. func (pt *PodTracker) LeastRecentlyUsed() string { @@ -145,9 +131,9 @@ func (pt *PodTracker) GetActivePodsCount() int { return pt.activePods.Length() } -func (pt *PodTracker) getPodKey(index int, vertexName string, vertexType string) string { +func (pt *PodTracker) getPodKey(index int, vertexName string) string { // podKey is used as a unique identifier for the pod, it is used by worker to determine the count of processed messages of the pod. - return strings.Join([]string{pt.pipeline.Name, vertexName, fmt.Sprintf("%d", index), vertexType}, PodInfoSeparator) + return strings.Join([]string{pt.pipeline.Name, vertexName, fmt.Sprintf("%d", index)}, PodInfoSeparator) } func (pt *PodTracker) isActive(vertexName, podName string) bool { diff --git a/pkg/daemon/server/service/rater/pod_tracker_test.go b/pkg/daemon/server/service/rater/pod_tracker_test.go index 95aaff5e6f..203645e9cc 100644 --- a/pkg/daemon/server/service/rater/pod_tracker_test.go +++ b/pkg/daemon/server/service/rater/pod_tracker_test.go @@ -118,8 +118,8 @@ func TestPodTracker_Start(t *testing.T) { cancel() wg.Wait() - assert.Equal(t, "p*v*0*other", tracker.LeastRecentlyUsed()) - assert.Equal(t, "p*v*1*other", tracker.LeastRecentlyUsed()) - assert.Equal(t, true, tracker.IsActive("p*v*4*other")) - assert.Equal(t, false, tracker.IsActive("p*v*5*other")) + assert.Equal(t, "p*v*0", tracker.LeastRecentlyUsed()) + assert.Equal(t, "p*v*1", tracker.LeastRecentlyUsed()) + assert.Equal(t, true, tracker.IsActive("p*v*4")) + assert.Equal(t, false, tracker.IsActive("p*v*5")) } diff --git a/pkg/daemon/server/service/rater/rater.go b/pkg/daemon/server/service/rater/rater.go index d5da3f69a2..d3060cd13d 100644 --- a/pkg/daemon/server/service/rater/rater.go +++ b/pkg/daemon/server/service/rater/rater.go @@ -28,6 +28,7 @@ import ( "go.uber.org/zap" "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" sharedqueue "github.com/numaproj/numaflow/pkg/shared/queue" ) @@ -133,15 +134,14 @@ func (r *Rater) monitorOnePod(ctx context.Context, key string, worker int) error log := logging.FromContext(ctx).With("worker", fmt.Sprint(worker)).With("podKey", key) log.Debugf("Working on key: %s", key) podInfo := strings.Split(key, PodInfoSeparator) - if len(podInfo) != 4 { + if len(podInfo) != 3 { return fmt.Errorf("invalid key %q", key) } vertexName := podInfo[1] - vertexType := podInfo[3] podName := strings.Join([]string{podInfo[0], podInfo[1], podInfo[2]}, "-") var podReadCount *PodReadCount if r.podTracker.IsActive(key) { - podReadCount = r.getPodReadCounts(vertexName, vertexType, podName) + podReadCount = r.getPodReadCounts(vertexName, podName) if podReadCount == nil { log.Debugf("Failed retrieving total podReadCount for pod %s", podName) } @@ -216,17 +216,8 @@ func sleep(ctx context.Context, duration time.Duration) { // getPodReadCounts returns the total number of messages read by the pod // since a pod can read from multiple partitions, we will return a map of partition to read count. -func (r *Rater) getPodReadCounts(vertexName, vertexType, podName string) *PodReadCount { - metricNames := map[string]string{ - keyVertexTypeReduce: "reduce_isb_reader_data_read", - keyVertexTypeSource: "source_forwarder_read_total", - keyVertexTypeSink: "sink_forwarder_data_read", - } - - readTotalMetricName, ok := metricNames[vertexType] - if !ok { - readTotalMetricName = "forwarder_data_read" - } +func (r *Rater) getPodReadCounts(vertexName, podName string) *PodReadCount { + readTotalMetricName := "forwarder_data_read_total" // scrape the read total metric from pod metric port url := fmt.Sprintf("https://%s.%s.%s.svc:%v/metrics", podName, r.pipeline.Name+"-"+vertexName+"-headless", r.pipeline.Namespace, v1alpha1.VertexMetricsPort) @@ -250,7 +241,7 @@ func (r *Rater) getPodReadCounts(vertexName, vertexType, podName string) *PodRea for _, ele := range metricsList { var partitionName string for _, label := range ele.Label { - if label.GetName() == "partition_name" { + if label.GetName() == metrics.LabelPartitionName { partitionName = label.GetValue() break } diff --git a/pkg/daemon/server/service/rater/rater_test.go b/pkg/daemon/server/service/rater/rater_test.go index 8520050c69..b33dbf8e4f 100644 --- a/pkg/daemon/server/service/rater/rater_test.go +++ b/pkg/daemon/server/service/rater/rater_test.go @@ -48,9 +48,9 @@ func (m *raterMockHttpClient) Get(url string) (*http.Response, error) { // the test uses an abstract vertex without specifying vertex type, meaning it's neither source nor reduce, // hence the default forwarder metric name "forwarder_data_read" is used to retrieve the metric Body: io.NopCloser(bytes.NewReader([]byte(fmt.Sprintf(` -# HELP forwarder_data_read Total number of Messages Read -# TYPE forwarder_data_read counter -forwarder_data_read{buffer="input",pipeline="simple-pipeline",vertex="input",partition_name="p-v-0"} %d +# HELP forwarder_data_read_total Total number of Messages Read +# TYPE forwarder_data_read_total counter +forwarder_data_read_total{buffer="input",pipeline="simple-pipeline",vertex="input",replica="0",partition_name="p-v-0"} %d `, m.podOneCount))))} return resp, nil } else if url == "https://p-v-1.p-v-headless.default.svc:2469/metrics" { @@ -58,9 +58,9 @@ forwarder_data_read{buffer="input",pipeline="simple-pipeline",vertex="input",par resp := &http.Response{ StatusCode: 200, Body: io.NopCloser(bytes.NewReader([]byte(fmt.Sprintf(` -# HELP forwarder_data_read Total number of Messages Read -# TYPE forwarder_data_read counter -forwarder_data_read{buffer="input",pipeline="simple-pipeline",vertex="input", partition_name="p-v-1"} %d +# HELP forwarder_data_read_total Total number of Messages Read +# TYPE forwarder_data_read_total counter +forwarder_data_read_total{buffer="input",pipeline="simple-pipeline",vertex="input",replica="0",partition_name="p-v-1"} %d `, m.podTwoCount))))} return resp, nil } else { diff --git a/pkg/forward/forward.go b/pkg/forward/forward.go index ef2bc47dd0..d0b6bd81d7 100644 --- a/pkg/forward/forward.go +++ b/pkg/forward/forward.go @@ -24,6 +24,7 @@ import ( "errors" "fmt" "math" + "strconv" "sync" "time" @@ -58,10 +59,11 @@ type InterStepDataForward struct { mapStreamUDF applier.MapStreamApplier wmFetcher fetch.Fetcher // wmPublishers stores the vertex to publisher mapping - wmPublishers map[string]publish.Publisher - opts options - vertexName string - pipelineName string + wmPublishers map[string]publish.Publisher + opts options + vertexName string + pipelineName string + vertexReplica int32 // idleManager manages the idle watermark status. idleManager wmb.IdleManager // wmbChecker checks if the idle watermark is valid when the len(readMessage) is 0. @@ -71,7 +73,7 @@ type InterStepDataForward struct { // NewInterStepDataForward creates an inter-step forwarder. func NewInterStepDataForward( - vertex *dfv1.Vertex, + vertexInstance *dfv1.VertexInstance, fromStep isb.BufferReader, toSteps map[string][]isb.BufferWriter, fsd ToWhichStepDecider, @@ -102,10 +104,11 @@ func NewInterStepDataForward( wmFetcher: fetchWatermark, wmPublishers: publishWatermark, // should we do a check here for the values not being null? - vertexName: vertex.Spec.Name, - pipelineName: vertex.Spec.PipelineName, - idleManager: idleManager, - wmbChecker: wmb.NewWMBChecker(2), // TODO: make configurable + vertexName: vertexInstance.Vertex.Spec.Name, + pipelineName: vertexInstance.Vertex.Spec.PipelineName, + vertexReplica: vertexInstance.Replica, + idleManager: idleManager, + wmbChecker: wmb.NewWMBChecker(2), // TODO: make configurable Shutdown: Shutdown{ rwlock: new(sync.RWMutex), }, @@ -197,7 +200,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { isdf.opts.logger.Debugw("Read from buffer", zap.String("bufferFrom", isdf.fromBufferPartition.GetName()), zap.Int64("length", int64(len(readMessages)))) if err != nil { isdf.opts.logger.Warnw("failed to read fromBufferPartition", zap.Error(err)) - readMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() + metrics.ReadMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() } // process only if we have any read messages. There is a natural looping here if there is an internal error while @@ -241,8 +244,8 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { dataMessages = append(dataMessages, m) } } - readMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(dataMessages))) - totalMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readMessages))) + metrics.ReadDataMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(dataMessages))) + metrics.ReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readMessages))) // fetch watermark if available // TODO: make it async (concurrent and wait later) @@ -281,7 +284,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // send to map UDF only the data messages for idx, m := range dataMessages { // emit message size metric - readBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(m.Payload))) + metrics.ReadBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(m.Payload))) // assign watermark to the message m.Watermark = time.Time(processorWM) // send map UDF processing work to the channel @@ -294,7 +297,7 @@ 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{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Observe(float64(time.Since(concurrentUDFProcessingStart).Microseconds())) + metrics.ConcurrentUDFProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(concurrentUDFProcessingStart).Microseconds())) // map UDF processing is done. // let's figure out which vertex to send the results to. @@ -303,7 +306,7 @@ func (isdf *InterStepDataForward) forwardAChunk(ctx context.Context) { // look for errors in udf processing, if we see even 1 error NoAck all messages // then return. Handling partial retrying is not worth ATM. if m.udfError != nil { - udfError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() + metrics.UDFError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() isdf.opts.logger.Errorw("failed to applyUDF", zap.Error(m.udfError)) // As there's no partial failure, non-ack all the readOffsets isdf.fromBufferPartition.NoAck(ctx, readOffsets) @@ -387,13 +390,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{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessageError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) return } - ackMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) // ProcessingTimes of the entire forwardAChunk - forwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.ForwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } // streamMessage streams the data messages to the next step. @@ -421,14 +424,14 @@ func (isdf *InterStepDataForward) streamMessage( // send to map UDF only the data messages // emit message size metric - readBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}). + metrics.ReadBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}). Add(float64(len(dataMessages[0].Payload))) // assign watermark to the message dataMessages[0].Watermark = time.Time(processorWM) // process the mapStreamUDF and get the result start := time.Now() - udfReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() + metrics.UDFReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() writeMessageCh := make(chan isb.WriteMessage) errs, ctx := errgroup.WithContext(ctx) @@ -443,7 +446,7 @@ func (isdf *InterStepDataForward) streamMessage( // add vertex name to the ID, since multiple vertices can publish to the same vertex and we need uniqueness across them writeMessage.ID = fmt.Sprintf("%s-%s-%d", dataMessages[0].ReadOffset.String(), isdf.vertexName, msgIndex) msgIndex += 1 - udfWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(1)) + metrics.UDFWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(1)) // update toBuffers if err := isdf.whereToStep(&writeMessage, messageToStep, dataMessages[0]); err != nil { @@ -466,18 +469,18 @@ func (isdf *InterStepDataForward) streamMessage( // look for errors in udf processing, if we see even 1 error NoAck all messages // then return. Handling partial retrying is not worth ATM. if err := errs.Wait(); err != nil { - udfError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, - metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() + metrics.UDFError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() // We do not retry as we are streaming if ok, _ := isdf.IsShuttingDown(); ok { isdf.opts.logger.Errorw("mapUDF.Apply, Stop called while stuck on an internal error", zap.Error(err)) - platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() } return nil, fmt.Errorf("failed to applyUDF, error: %w", err) } - udfProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, - metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.UDFProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } else { // Even not data messages, forward the message to the edge buffer (could be multiple edges) var err error @@ -587,10 +590,10 @@ func (isdf *InterStepDataForward) writeToBuffer(ctx context.Context, toBufferPar needRetry = true // we retry only failed messages failedMessages = append(failedMessages, msg) - writeMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() + metrics.WriteMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() // a shutdown can break the blocking loop caused due to InternalErr if ok, _ := isdf.IsShuttingDown(); ok { - platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return writeOffsets, fmt.Errorf("writeToBuffer failed, Stop called while stuck on an internal error with failed messages:%d, %v", len(failedMessages), errs) } } @@ -620,10 +623,10 @@ func (isdf *InterStepDataForward) writeToBuffer(ctx context.Context, toBufferPar } } - dropMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) - dropBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) - writeMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) - writeBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) + metrics.DropMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) + metrics.DropBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) + metrics.WriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) + metrics.WriteBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) return writeOffsets, nil } @@ -631,12 +634,12 @@ func (isdf *InterStepDataForward) writeToBuffer(ctx context.Context, toBufferPar func (isdf *InterStepDataForward) concurrentApplyUDF(ctx context.Context, readMessagePair <-chan *readWriteMessagePair) { for message := range readMessagePair { start := time.Now() - udfReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() + metrics.UDFReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Inc() writeMessages, err := isdf.applyUDF(ctx, message.readMessage) - udfWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(writeMessages))) + metrics.UDFWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Add(float64(len(writeMessages))) message.writeMessages = append(message.writeMessages, writeMessages...) message.udfError = err - udfProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.fromBufferPartition.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.UDFProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } } @@ -655,7 +658,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("mapUDF.Apply, Stop called while stuck on an internal error", zap.Error(err)) - platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return nil, err } continue @@ -682,7 +685,7 @@ func (isdf *InterStepDataForward) whereToStep(writeMessage *isb.WriteMessage, me // 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{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeMapUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return err } return err diff --git a/pkg/forward/forward_test.go b/pkg/forward/forward_test.go index e386c0d802..55608d69d6 100644 --- a/pkg/forward/forward_test.go +++ b/pkg/forward/forward_test.go @@ -24,15 +24,15 @@ import ( "testing" "time" - "go.uber.org/goleak" - "github.com/prometheus/client_golang/prometheus/testutil" "github.com/stretchr/testify/assert" + "go.uber.org/goleak" dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/kvs" "github.com/numaproj/numaflow/pkg/shared/logging" udfapplier "github.com/numaproj/numaflow/pkg/udf/rpc" @@ -130,6 +130,10 @@ func TestNewInterStepDataForward(t *testing.T) { Name: "testVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -137,7 +141,7 @@ func TestNewInterStepDataForward(t *testing.T) { writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -218,6 +222,11 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} toVertexWmStores := buildWatermarkStores(toSteps) publishWatermark, otStores := buildPublisherMapAndOTStoreFromWmStores(toSteps, toVertexWmStores) @@ -234,7 +243,7 @@ func TestNewInterStepDataForward(t *testing.T) { } }() - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, &myForwardToAllTest{}, &myForwardToAllTest{}, &myForwardToAllTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, &myForwardToAllTest{}, &myForwardToAllTest{}, &myForwardToAllTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -378,6 +387,11 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} toVertexWmStores := buildWatermarkStores(toSteps) publishWatermark, otStores := buildPublisherMapAndOTStoreFromWmStores(toSteps, toVertexWmStores) @@ -394,7 +408,7 @@ func TestNewInterStepDataForward(t *testing.T) { } }() - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardDropTest{}, myForwardDropTest{}, myForwardDropTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardDropTest{}, myForwardDropTest{}, myForwardDropTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -550,6 +564,11 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} toVertexWmStores := buildWatermarkStores(toSteps) publishWatermark, otStores := buildPublisherMapAndOTStoreFromWmStores(toSteps, toVertexWmStores) @@ -566,7 +585,7 @@ func TestNewInterStepDataForward(t *testing.T) { } }() - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -689,13 +708,18 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() writeMessages := testutils.BuildTestWriteMessages(4*batchSize, testStartTime) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardApplyUDFErrTest{}, myForwardApplyUDFErrTest{}, myForwardApplyUDFErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardApplyUDFErrTest{}, myForwardApplyUDFErrTest{}, myForwardApplyUDFErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -732,8 +756,13 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.True(t, to1.IsEmpty()) @@ -768,8 +797,13 @@ func TestNewInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardInternalErrTest{}, myForwardInternalErrTest{}, myForwardInternalErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardInternalErrTest{}, myForwardInternalErrTest{}, myForwardInternalErrTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -881,6 +915,11 @@ func TestNewInterStepDataForwardIdleWatermark(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + ctrlMessage := []isb.Message{{Header: isb.Header{Kind: isb.WMB}}} writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) @@ -900,7 +939,7 @@ func TestNewInterStepDataForwardIdleWatermark(t *testing.T) { } }() - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(2)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(2)) assert.NoError(t, err) assert.False(t, to1.IsFull()) assert.True(t, to1.IsEmpty()) @@ -1048,6 +1087,11 @@ func TestNewInterStepDataForwardIdleWatermark_Reset(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) fetchWatermark := &testWMBFetcher{WMBTestSameHeadWMB: true} @@ -1073,7 +1117,7 @@ func TestNewInterStepDataForwardIdleWatermark_Reset(t *testing.T) { } }() - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(2)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(2)) assert.NoError(t, err) assert.False(t, to1.IsFull()) assert.True(t, to1.IsEmpty()) @@ -1287,6 +1331,11 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { Name: "receivingVertex", }, }} + + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -1295,7 +1344,7 @@ func TestInterStepDataForwardSinglePartition(t *testing.T) { _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) // create a forwarder - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, mySourceForwardTest{}, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, mySourceForwardTest{}, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) assert.NoError(t, err) assert.False(t, to1.IsFull()) assert.True(t, to1.IsEmpty()) @@ -1333,6 +1382,11 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { Name: "receivingVertex", }, }} + + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -1341,7 +1395,7 @@ func TestInterStepDataForwardMultiplePartition(t *testing.T) { _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) // create a forwarder - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) assert.NoError(t, err) assert.False(t, to11.IsFull()) assert.False(t, to12.IsFull()) @@ -1436,8 +1490,13 @@ func TestWriteToBuffer(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(value.batchSize), WithUDFStreaming(value.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardTest{}, myForwardTest{}, myForwardTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(value.batchSize), WithUDFStreaming(value.streamEnabled)) assert.NoError(t, err) assert.False(t, buffer.IsFull()) assert.True(t, buffer.IsEmpty()) @@ -1585,14 +1644,14 @@ func (f myForwardApplyUDFErrTest) ApplyMapStream(_ context.Context, _ *isb.ReadM func validateMetrics(t *testing.T, batchSize int64) { metadata := ` - # HELP forwarder_data_read Total number of Data Messages Read - # TYPE forwarder_data_read counter + # HELP forwarder_data_read_total Total number of Data Messages Read + # TYPE forwarder_data_read_total counter ` expected := ` - forwarder_data_read{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` + forwarder_data_read_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` ` - err := testutil.CollectAndCompare(readMessagesCount, strings.NewReader(metadata+expected), "forwarder_data_read") + err := testutil.CollectAndCompare(metrics.ReadDataMessagesCount, strings.NewReader(metadata+expected), "forwarder_data_read_total") if err != nil { t.Errorf("unexpected collecting result:\n%s", err) } @@ -1604,17 +1663,17 @@ func validateMetrics(t *testing.T, batchSize int64) { var writeExpected string if batchSize > 1 { writeExpected = ` - forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` - forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` + forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` + forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` ` } else { writeExpected = ` - forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` - forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(0)) + ` + forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` + forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%f", float64(0)) + ` ` } - err = testutil.CollectAndCompare(writeMessagesCount, strings.NewReader(writeMetadata+writeExpected), "forwarder_write_total") + err = testutil.CollectAndCompare(metrics.WriteMessagesCount, strings.NewReader(writeMetadata+writeExpected), "forwarder_write_total") if err != nil { t.Errorf("unexpected collecting result:\n%s", err) } @@ -1624,19 +1683,19 @@ func validateMetrics(t *testing.T, batchSize int64) { # TYPE forwarder_ack_total counter ` ackExpected := ` - forwarder_ack_total{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%d", batchSize) + ` + forwarder_ack_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="MapUDF"} ` + fmt.Sprintf("%d", batchSize) + ` ` - err = testutil.CollectAndCompare(ackMessagesCount, strings.NewReader(ackMetadata+ackExpected), "forwarder_ack_total") + err = testutil.CollectAndCompare(metrics.AckMessagesCount, strings.NewReader(ackMetadata+ackExpected), "forwarder_ack_total") if err != nil { t.Errorf("unexpected collecting result:\n%s", err) } } func metricsReset() { - readMessagesCount.Reset() - writeMessagesCount.Reset() - ackMessagesCount.Reset() + metrics.ReadDataMessagesCount.Reset() + metrics.WriteMessagesCount.Reset() + metrics.AckMessagesCount.Reset() } func buildWatermarkStores(toBuffers map[string][]isb.BufferWriter) map[string]wmstore.WatermarkStore { diff --git a/pkg/forward/metrics.go b/pkg/forward/metrics.go deleted file mode 100644 index d705852f49..0000000000 --- a/pkg/forward/metrics.go +++ /dev/null @@ -1,153 +0,0 @@ -/* -Copyright 2022 The Numaproj Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package forward - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/numaproj/numaflow/pkg/metrics" -) - -// totalMessagesCount is used to indicate the number of total messages read -var totalMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "forwarder", - Name: "total_read", - Help: "Total number of Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readMessagesCount is used to indicate the number of data messages read -var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "forwarder", - Name: "data_read", - Help: "Total number of Data Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropMessagesCount is used to indicate the number of messages dropped -var dropMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "forwarder", - Name: "drop_total", - Help: "Total number of Messages Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropBytesCount is to indicate the number of bytes dropped -var dropBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "forwarder", - Name: "drop_bytes_total", - Help: "Total number of Bytes Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline}) - -// forwardAChunkProcessingTime is a histogram to Observe forwardAChunk Processing times as a whole -var forwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "forwarder", - 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, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// udfProcessingTime is a histogram to Observe UDF Processing times as a whole -var udfProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "forwarder", - Name: "udf_processing_time", - Help: "Processing times of UDF (100 microseconds to 15 minutes)", - Buckets: prometheus.ExponentialBucketsRange(100, 60000000*15, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// concurrentUDFProcessingTime is a histogram to Observe UDF Processing times as a whole -var concurrentUDFProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "forwarder", - Name: "concurrent_udf_processing_time", - Help: "Processing times of Concurrent UDF (100 microseconds to 20 minutes)", - Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// 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 by UDF", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// udfWriteMessagesCount is used to indicate the number of messages written by UDF -var udfWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "forwarder", - Name: "udf_write_total", - Help: "Total number of Messages Written by UDF", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) diff --git a/pkg/forward/shutdown_test.go b/pkg/forward/shutdown_test.go index 00fb915d91..c9c5ee70bd 100644 --- a/pkg/forward/shutdown_test.go +++ b/pkg/forward/shutdown_test.go @@ -84,8 +84,13 @@ func TestInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, myShutdownTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, myShutdownTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) stopped := f.Start() // write some data but buffer is not full even though we are not reading @@ -117,8 +122,13 @@ func TestInterStepDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewInterStepDataForward(vertex, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, myShutdownTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) + f, err := NewInterStepDataForward(vertexInstance, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, myShutdownTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize), WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) stopped := f.Start() // write some data such that the fromBufferPartition can be empty, that is toBuffer gets full diff --git a/pkg/isb/stores/jetstream/writer_test.go b/pkg/isb/stores/jetstream/writer_test.go index ca345c6593..42dcf9778f 100644 --- a/pkg/isb/stores/jetstream/writer_test.go +++ b/pkg/isb/stores/jetstream/writer_test.go @@ -125,6 +125,11 @@ func TestForwarderJetStreamBuffer(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + // Forwarder logic tested here with a jetstream read and write bufferReader, err := NewJetStreamBufferReader(ctx, defaultJetStreamClient, streamName, streamName, streamName, defaultPartitionIdx) assert.NoError(t, err) @@ -140,7 +145,7 @@ func TestForwarderJetStreamBuffer(t *testing.T) { } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, myForwardJetStreamTest{}, myForwardJetStreamTest{}, myForwardJetStreamTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), forward.WithReadBatchSize(tt.batchSize), forward.WithUDFStreaming(tt.streamEnabled)) + f, err := forward.NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardJetStreamTest{}, myForwardJetStreamTest{}, myForwardJetStreamTest{}, fetchWatermark, publishWatermark, wmb.NewIdleManager(len(toSteps)), forward.WithReadBatchSize(tt.batchSize), forward.WithUDFStreaming(tt.streamEnabled)) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/isb/stores/redis/read_test.go b/pkg/isb/stores/redis/read_test.go index 0809614f6d..079a560250 100644 --- a/pkg/isb/stores/redis/read_test.go +++ b/pkg/isb/stores/redis/read_test.go @@ -123,6 +123,11 @@ func TestRedisCheckBacklog(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + rqw, _ := NewBufferWrite(ctx, client, "toStream", "toGroup", defaultPartitionIdx, redisclient.WithInfoRefreshInterval(2*time.Millisecond), redisclient.WithLagDuration(time.Minute)).(*BufferWrite) err = client.CreateStreamGroup(ctx, rqw.GetStreamName(), "toGroup", redisclient.ReadFromEarliest) assert.NoError(t, err) @@ -134,7 +139,7 @@ func TestRedisCheckBacklog(t *testing.T) { } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager(), forward.WithReadBatchSize(10)) + f, err := forward.NewInterStepDataForward(vertexInstance, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager(), forward.WithReadBatchSize(10)) stopped := f.Start() // validate the length of the toStep stream. @@ -332,8 +337,13 @@ func (suite *ReadWritePerformance) SetupSuite() { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - isdf, _ := forward.NewInterStepDataForward(vertex, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) + isdf, _ := forward.NewInterStepDataForward(vertexInstance, rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) suite.ctx = ctx suite.rclient = client @@ -417,12 +427,17 @@ func (suite *ReadWritePerformance) TestReadWriteLatencyPipelining() { Name: "testVertex", }, }} + + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } toSteps := map[string][]isb.BufferWriter{ "to1": {suite.rqw}, } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - suite.isdf, _ = forward.NewInterStepDataForward(vertex, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) + suite.isdf, _ = forward.NewInterStepDataForward(vertexInstance, suite.rqr, toSteps, forwardReadWritePerformance{}, forwardReadWritePerformance{}, forwardReadWritePerformance{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) 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 53ac8915af..a412d24758 100644 --- a/pkg/isb/stores/redis/write_test.go +++ b/pkg/isb/stores/redis/write_test.go @@ -402,8 +402,13 @@ func TestNewInterStepDataForwardRedis(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, myForwardRedisTest{}, myForwardRedisTest{}, myForwardRedisTest{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) + f, err := forward.NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardRedisTest{}, myForwardRedisTest{}, myForwardRedisTest{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -446,8 +451,13 @@ func TestReadTimeout(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := forward.NewInterStepDataForward(vertex, fromStep, toSteps, myForwardRedisTest{}, myForwardRedisTest{}, myForwardRedisTest{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) + f, err := forward.NewInterStepDataForward(vertexInstance, fromStep, toSteps, myForwardRedisTest{}, myForwardRedisTest{}, myForwardRedisTest{}, fetchWatermark, publishWatermark, wmb.NewNoOpIdleManager()) assert.NoError(t, err) stopped := f.Start() // Call stop to end the test as we have a blocking read. The forwarder is up and running with no messages written diff --git a/pkg/metrics/metrics.go b/pkg/metrics/metrics.go index 89b8ed2608..c1572e4cff 100644 --- a/pkg/metrics/metrics.go +++ b/pkg/metrics/metrics.go @@ -17,280 +17,243 @@ limitations under the License. package metrics import ( - "context" - "crypto/tls" - "fmt" - "net/http" - "net/http/pprof" - "os" - "time" - "github.com/prometheus/client_golang/prometheus" "github.com/prometheus/client_golang/prometheus/promauto" - "github.com/prometheus/client_golang/prometheus/promhttp" - "go.uber.org/zap" - - dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" - "github.com/numaproj/numaflow/pkg/isb" - "github.com/numaproj/numaflow/pkg/shared/logging" - sharedqueue "github.com/numaproj/numaflow/pkg/shared/queue" - sharedtls "github.com/numaproj/numaflow/pkg/shared/tls" - "github.com/numaproj/numaflow/pkg/shared/util" ) const ( LabelPipeline = "pipeline" LabelVertex = "vertex" - LabelPeriod = "period" LabelVertexReplicaIndex = "replica" + LabelVertexType = "vertex_type" LabelPartitionName = "partition_name" - VertexPendingMessages = "vertex_pending_messages" + LabelReason = "reason" +) + +// Generic forwarder metrics +var ( + // ReadMessagesCount is used to indicate the number of total messages read + ReadMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "read_total", + Help: "Total number of Messages Read", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // ReadDataMessagesCount is used to indicate the number of data messages read + ReadDataMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "data_read_total", + Help: "Total number of Data Messages Read", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // ReadBytesCount is to indicate the number of bytes read + ReadBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "read_bytes_total", + Help: "Total number of bytes read", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // ReadMessagesError is used to indicate the number of errors messages read + ReadMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "read_error_total", + Help: "Total number of Read Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // WriteMessagesCount is used to indicate the number of messages written + WriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "write_total", + Help: "Total number of Messages Written", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // WriteBytesCount is to indicate the number of bytes written + WriteBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "write_bytes_total", + Help: "Total number of bytes written", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // WriteMessagesError is used to indicate the number of errors messages written + WriteMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "write_error_total", + Help: "Total number of Write Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // DropMessagesCount is used to indicate the number of messages dropped + DropMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "drop_total", + Help: "Total number of Messages Dropped", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // DropBytesCount is to indicate the number of bytes dropped + DropBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "drop_bytes_total", + Help: "Total number of Bytes Dropped", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // AckMessagesCount is used to indicate the number of messages acknowledged + AckMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "ack_total", + Help: "Total number of Messages Acknowledged", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // AckMessageError is used to indicate the errors in the number of messages acknowledged + AckMessageError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "ack_error_total", + Help: "Total number of Acknowledged Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // UDFError is used to indicate the number of UDF errors + UDFError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "udf_error_total", + Help: "Total number of UDF Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex}) + + // PlatformError is used to indicate the number of Internal/Platform errors + PlatformError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "platform_error_total", + Help: "Total number of platform Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex}) + + // ForwardAChunkProcessingTime is a histogram to Observe forwardAChunk Processing times as a whole + ForwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "forwarder", + 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, 10), + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex}) + + // UDFProcessingTime is a histogram to Observe UDF Processing times as a whole + UDFProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "forwarder", + Name: "udf_processing_time", + Help: "Processing times of UDF (100 microseconds to 15 minutes)", + Buckets: prometheus.ExponentialBucketsRange(100, 60000000*15, 10), + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex}) + + // ConcurrentUDFProcessingTime is a histogram to Observe UDF Processing times as a whole + ConcurrentUDFProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "forwarder", + Name: "concurrent_udf_processing_time", + Help: "Processing times of Concurrent UDF (100 microseconds to 20 minutes)", + Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 10), + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex}) + + // UDFReadMessagesCount is used to indicate the number of messages read by UDF + UDFReadMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "udf_read_total", + Help: "Total number of Messages Read by UDF", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) + + // UDFWriteMessagesCount is used to indicate the number of messages written by UDF + UDFWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "forwarder", + Name: "udf_write_total", + Help: "Total number of Messages Written by UDF", + }, []string{LabelVertex, LabelPipeline, LabelVertexType, LabelVertexReplicaIndex, LabelPartitionName}) ) +// Source forwarder specific metrics var ( - pending = promauto.NewGaugeVec(prometheus.GaugeOpts{ - Name: VertexPendingMessages, - Help: "Average pending messages in the last period of seconds. It is the pending messages of a vertex, not a pod.", - }, []string{LabelPipeline, LabelVertex, LabelPeriod, LabelPartitionName}) + // SourceTransformerError is used to indicate the number of source transformer errors + SourceTransformerError = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "source_forwarder", + Name: "transformer_error_total", + Help: "Total number of source transformer Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelPartitionName}) + + // SourceTransformerProcessingTime is a histogram to Observe Source Transformer Processing times as a whole + SourceTransformerProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "source_forwarder", + Name: "transformer_processing_time", + Help: "Processing times of source transformer (100 microseconds to 15 minutes)", + Buckets: prometheus.ExponentialBucketsRange(100, 60000000*15, 10), + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelPartitionName}) + + // SourceTransformerConcurrentProcessingTime is a histogram to Observe Source Transformer Processing times as a whole + SourceTransformerConcurrentProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "source_forwarder", + Name: "concurrent_transformer_processing_time", + Help: "Processing times of Concurrent source transformer (100 microseconds to 20 minutes)", + Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 10), + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelPartitionName}) + + // SourceTransformerReadMessagesCount is used to indicate the number of messages read by source transformer + SourceTransformerReadMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "source_forwarder", + Name: "transformer_read_total", + Help: "Total number of Messages Read by source transformer", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelPartitionName}) - // fixedLookbackSeconds Always expose metrics of following lookback seconds (1m, 5m, 15m) - fixedLookbackSeconds = map[string]int64{"1m": 60, "5m": 300, "15m": 900} + // SourceTransformerWriteMessagesCount is used to indicate the number of messages written by source transformer + SourceTransformerWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "source_forwarder", + Name: "transformer_write_total", + Help: "Total number of Messages Written by source transformer", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelPartitionName}) ) -// timestampedPending is a helper struct to wrap a pending number and timestamp pair -type timestampedPending struct { - pending int64 - // timestamp in seconds - timestamp int64 -} - -// metricsServer runs an HTTP server to: -// 1. Expose metrics; -// 2. Serve an endpoint to execute health checks -type metricsServer struct { - vertex *dfv1.Vertex - lagReaders map[string]isb.LagReader - // lookbackSeconds is the look back seconds for pending calculation used for autoscaling - lookbackSeconds int64 - lagCheckingInterval time.Duration - refreshInterval time.Duration - // partitionPendingInfo stores a list of pending/timestamp(seconds) information for each partition - partitionPendingInfo map[string]*sharedqueue.OverflowQueue[timestampedPending] - // Functions that health check executes - healthCheckExecutors []func() error -} - -type Option func(*metricsServer) - -// WithLagReaders sets the lag readers -func WithLagReaders(r map[string]isb.LagReader) Option { - return func(m *metricsServer) { - m.lagReaders = r - } -} - -// WithRefreshInterval sets how often to refresh the pending information -func WithRefreshInterval(d time.Duration) Option { - return func(m *metricsServer) { - m.refreshInterval = d - } -} - -// WithLookbackSeconds sets lookback seconds for pending calculation -func WithLookbackSeconds(seconds int64) Option { - return func(m *metricsServer) { - m.lookbackSeconds = seconds - } -} - -// WithHealthCheckExecutor appends a health check executor -func WithHealthCheckExecutor(f func() error) Option { - return func(m *metricsServer) { - m.healthCheckExecutors = append(m.healthCheckExecutors, f) - } -} - -// NewMetricsOptions returns a metrics option list. -func NewMetricsOptions(ctx context.Context, vertex *dfv1.Vertex, healthCheckers []HealthChecker, readers []isb.BufferReader) []Option { - metricsOpts := []Option{ - WithLookbackSeconds(int64(vertex.Spec.Scale.GetLookbackSeconds())), - } - - if util.LookupEnvStringOr(dfv1.EnvHealthCheckDisabled, "false") != "true" { - for _, hc := range healthCheckers { - metricsOpts = append(metricsOpts, WithHealthCheckExecutor(func() error { - cctx, cancel := context.WithTimeout(ctx, 30*time.Second) - defer cancel() - return hc.IsHealthy(cctx) - })) - } - } - - lagReaders := make(map[string]isb.LagReader) - for _, reader := range readers { - if x, ok := reader.(isb.LagReader); ok { - lagReaders[reader.GetName()] = x - } - } - if len(lagReaders) > 0 { - metricsOpts = append(metricsOpts, WithLagReaders(lagReaders)) - } - return metricsOpts -} - -// NewMetricsServer returns a Prometheus metrics server instance, which can be used to start an HTTPS service to expose Prometheus metrics. -func NewMetricsServer(vertex *dfv1.Vertex, opts ...Option) *metricsServer { - m := new(metricsServer) - m.vertex = vertex - m.partitionPendingInfo = make(map[string]*sharedqueue.OverflowQueue[timestampedPending]) - m.refreshInterval = 5 * time.Second // Default refresh interval - m.lagCheckingInterval = 3 * time.Second // Default lag checking interval - m.lookbackSeconds = dfv1.DefaultLookbackSeconds // Default - for _, opt := range opts { - if opt != nil { - opt(m) - } - } - if m.lagReaders != nil { - for partitionName := range m.lagReaders { - m.partitionPendingInfo[partitionName] = sharedqueue.New[timestampedPending](1800) - - } - } - return m -} - -// Enqueue pending information -func (ms *metricsServer) buildupPendingInfo(ctx context.Context) { - if ms.lagReaders == nil { - return - } - log := logging.FromContext(ctx) - ticker := time.NewTicker(ms.lagCheckingInterval) - defer ticker.Stop() - for { - select { - case <-ctx.Done(): - return - case <-ticker.C: - for partitionName, lagReader := range ms.lagReaders { - if pending, err := lagReader.Pending(ctx); err != nil { - log.Errorw("Failed to get pending messages", zap.Error(err)) - } else { - if pending != isb.PendingNotAvailable { - ts := timestampedPending{pending: pending, timestamp: time.Now().Unix()} - ms.partitionPendingInfo[partitionName].Append(ts) - } - } - } - } - } -} - -// Expose pending metrics -func (ms *metricsServer) exposePendingMetrics(ctx context.Context) { - if ms.lagReaders == nil { - return - } - lookbackSecondsMap := map[string]int64{"default": ms.lookbackSeconds} // Metrics for autoscaling use key "default" - for k, v := range fixedLookbackSeconds { - lookbackSecondsMap[k] = v - } - ticker := time.NewTicker(ms.refreshInterval) - defer ticker.Stop() - for { - select { - case <-ticker.C: - if ms.lagReaders != nil { - for partitionName := range ms.lagReaders { - for n, i := range lookbackSecondsMap { - if p := ms.calculatePending(i, partitionName); p != isb.PendingNotAvailable { - pending.WithLabelValues(ms.vertex.Spec.PipelineName, ms.vertex.Spec.Name, n, partitionName).Set(float64(p)) - } - } - } - } - case <-ctx.Done(): - return - } - } -} - -// Calculate the avg pending of last seconds -func (ms *metricsServer) calculatePending(seconds int64, partitionName string) int64 { - result := isb.PendingNotAvailable - items := ms.partitionPendingInfo[partitionName].Items() - total := int64(0) - num := int64(0) - now := time.Now().Unix() - for i := len(items) - 1; i >= 0; i-- { - if now-items[i].timestamp < seconds { - total += items[i].pending - num++ - } else { - break - } - } - if num > 0 { - result = total / num - } - return result -} - -// Start function starts the HTTPS service to expose metrics, it returns a shutdown function and an error if any -func (ms *metricsServer) Start(ctx context.Context) (func(ctx context.Context) error, error) { - log := logging.FromContext(ctx) - log.Info("Generating self-signed certificate") - cer, err := sharedtls.GenerateX509KeyPair() - if err != nil { - return nil, fmt.Errorf("failed to generate cert: %w", err) - } - mux := http.NewServeMux() - mux.Handle("/metrics", promhttp.Handler()) - mux.HandleFunc("/readyz", func(w http.ResponseWriter, r *http.Request) { - w.WriteHeader(http.StatusNoContent) - }) - mux.HandleFunc("/livez", func(w http.ResponseWriter, r *http.Request) { - w.WriteHeader(http.StatusNoContent) - }) - mux.HandleFunc("/sidecar-livez", func(w http.ResponseWriter, r *http.Request) { - for _, ex := range ms.healthCheckExecutors { - if err := ex(); err != nil { - log.Errorw("Failed to execute sidecar health check", zap.Error(err)) - w.WriteHeader(http.StatusInternalServerError) - _, _ = w.Write([]byte(err.Error())) - return - } - } - w.WriteHeader(http.StatusNoContent) - }) - 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) - mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) - mux.HandleFunc("/debug/pprof/trace", pprof.Trace) - } else { - log.Info("Not enabling pprof debug endpoints") - } - - httpServer := &http.Server{ - Addr: fmt.Sprintf(":%d", dfv1.VertexMetricsPort), - Handler: mux, - TLSConfig: &tls.Config{Certificates: []tls.Certificate{*cer}, MinVersion: tls.VersionTLS12}, - } - // Buildup pending information - go ms.buildupPendingInfo(ctx) - // Expose pending metrics - go ms.exposePendingMetrics(ctx) - go func() { - log.Info("Starting metrics HTTPS server") - if err := httpServer.ListenAndServeTLS("", ""); err != nil && err != http.ErrServerClosed { - log.Fatalw("Failed to listen-and-server on HTTPS", zap.Error(err)) - } - log.Info("Metrics server shutdown") - }() - return httpServer.Shutdown, nil -} +// Reduce forwarder specific metrics +var ( + // ReduceDroppedMessagesCount is used to indicate the number of messages dropped + ReduceDroppedMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "reduce_data_forward", + Name: "dropped_total", + Help: "Total number of Messages Dropped", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex, LabelReason}) + + // PBQWriteErrorCount is used to indicate the number of errors while writing to pbq + PBQWriteErrorCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "reduce_pbq", + Name: "write_error_total", + Help: "Total number of PBQ Write Errors", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex}) + + // PBQWriteMessagesCount is used to indicate the number of messages written to pbq + PBQWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ + Subsystem: "reduce_pbq", + Name: "write_total", + Help: "Total number of Messages Written to PBQ", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex}) + + // PBQWriteTime pbq write latency + PBQWriteTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "reduce_pbq", + Name: "write_time", + Help: "Entry write time (1 to 5000 microseconds)", + Buckets: prometheus.ExponentialBucketsRange(1, 5000, 5), + }, []string{LabelPipeline, LabelVertex, LabelVertexReplicaIndex}) + + // ReduceProcessTime reduce ForwardTask processing latency + ReduceProcessTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "reduce_pnf", + Name: "process_time", + Help: "Reduce process time (1 to 1200000 milliseconds)", + Buckets: prometheus.ExponentialBucketsRange(1, 1200000, 5), + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex}) + + // ReduceForwardTime is used to indicate the time it took to forward the writeMessages + ReduceForwardTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ + Subsystem: "reduce_pnf", + Name: "forward_time", + Help: "Reduce forward time (1 to 100000 microseconds)", + Buckets: prometheus.ExponentialBucketsRange(1, 100000, 5), + }, []string{LabelPipeline, LabelVertex, LabelVertexReplicaIndex}) + + // ReducePartitionsInFlight is used to indicate the partitions in flight + ReducePartitionsInFlight = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Subsystem: "reduce_pnf", + Name: "partitions_inflight", + Help: "Total number of partitions in flight", + }, []string{LabelVertex, LabelPipeline, LabelVertexReplicaIndex}) +) diff --git a/pkg/metrics/metrics_server.go b/pkg/metrics/metrics_server.go new file mode 100644 index 0000000000..774db0dc33 --- /dev/null +++ b/pkg/metrics/metrics_server.go @@ -0,0 +1,291 @@ +/* +Copyright 2022 The Numaproj Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package metrics + +import ( + "context" + "crypto/tls" + "fmt" + "net/http" + "net/http/pprof" + "os" + "time" + + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "github.com/prometheus/client_golang/prometheus/promhttp" + "go.uber.org/zap" + + dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/isb" + "github.com/numaproj/numaflow/pkg/shared/logging" + sharedqueue "github.com/numaproj/numaflow/pkg/shared/queue" + sharedtls "github.com/numaproj/numaflow/pkg/shared/tls" + "github.com/numaproj/numaflow/pkg/shared/util" +) + +const ( + VertexPendingMessages = "vertex_pending_messages" + LabelPeriod = "period" +) + +var ( + pending = promauto.NewGaugeVec(prometheus.GaugeOpts{ + Name: VertexPendingMessages, + Help: "Average pending messages in the last period of seconds. It is the pending messages of a vertex, not a pod.", + }, []string{LabelPipeline, LabelVertex, LabelPeriod, LabelPartitionName}) + + // fixedLookbackSeconds Always expose metrics of following lookback seconds (1m, 5m, 15m) + fixedLookbackSeconds = map[string]int64{"1m": 60, "5m": 300, "15m": 900} +) + +// timestampedPending is a helper struct to wrap a pending number and timestamp pair +type timestampedPending struct { + pending int64 + // timestamp in seconds + timestamp int64 +} + +// metricsServer runs an HTTP server to: +// 1. Expose metrics; +// 2. Serve an endpoint to execute health checks +type metricsServer struct { + vertex *dfv1.Vertex + lagReaders map[string]isb.LagReader + // lookbackSeconds is the look back seconds for pending calculation used for autoscaling + lookbackSeconds int64 + lagCheckingInterval time.Duration + refreshInterval time.Duration + // partitionPendingInfo stores a list of pending/timestamp(seconds) information for each partition + partitionPendingInfo map[string]*sharedqueue.OverflowQueue[timestampedPending] + // Functions that health check executes + healthCheckExecutors []func() error +} + +type Option func(*metricsServer) + +// WithLagReaders sets the lag readers +func WithLagReaders(r map[string]isb.LagReader) Option { + return func(m *metricsServer) { + m.lagReaders = r + } +} + +// WithRefreshInterval sets how often to refresh the pending information +func WithRefreshInterval(d time.Duration) Option { + return func(m *metricsServer) { + m.refreshInterval = d + } +} + +// WithLookbackSeconds sets lookback seconds for pending calculation +func WithLookbackSeconds(seconds int64) Option { + return func(m *metricsServer) { + m.lookbackSeconds = seconds + } +} + +// WithHealthCheckExecutor appends a health check executor +func WithHealthCheckExecutor(f func() error) Option { + return func(m *metricsServer) { + m.healthCheckExecutors = append(m.healthCheckExecutors, f) + } +} + +// NewMetricsOptions returns a metrics option list. +func NewMetricsOptions(ctx context.Context, vertex *dfv1.Vertex, healthCheckers []HealthChecker, readers []isb.BufferReader) []Option { + metricsOpts := []Option{ + WithLookbackSeconds(int64(vertex.Spec.Scale.GetLookbackSeconds())), + } + + if util.LookupEnvStringOr(dfv1.EnvHealthCheckDisabled, "false") != "true" { + for _, hc := range healthCheckers { + metricsOpts = append(metricsOpts, WithHealthCheckExecutor(func() error { + cctx, cancel := context.WithTimeout(ctx, 30*time.Second) + defer cancel() + return hc.IsHealthy(cctx) + })) + } + } + + lagReaders := make(map[string]isb.LagReader) + for _, reader := range readers { + if x, ok := reader.(isb.LagReader); ok { + lagReaders[reader.GetName()] = x + } + } + if len(lagReaders) > 0 { + metricsOpts = append(metricsOpts, WithLagReaders(lagReaders)) + } + return metricsOpts +} + +// NewMetricsServer returns a Prometheus metrics server instance, which can be used to start an HTTPS service to expose Prometheus metrics. +func NewMetricsServer(vertex *dfv1.Vertex, opts ...Option) *metricsServer { + m := new(metricsServer) + m.vertex = vertex + m.partitionPendingInfo = make(map[string]*sharedqueue.OverflowQueue[timestampedPending]) + m.refreshInterval = 5 * time.Second // Default refresh interval + m.lagCheckingInterval = 3 * time.Second // Default lag checking interval + m.lookbackSeconds = dfv1.DefaultLookbackSeconds // Default + for _, opt := range opts { + if opt != nil { + opt(m) + } + } + if m.lagReaders != nil { + for partitionName := range m.lagReaders { + m.partitionPendingInfo[partitionName] = sharedqueue.New[timestampedPending](1800) + + } + } + return m +} + +// Enqueue pending information +func (ms *metricsServer) buildupPendingInfo(ctx context.Context) { + if ms.lagReaders == nil { + return + } + log := logging.FromContext(ctx) + ticker := time.NewTicker(ms.lagCheckingInterval) + defer ticker.Stop() + for { + select { + case <-ctx.Done(): + return + case <-ticker.C: + for partitionName, lagReader := range ms.lagReaders { + if pending, err := lagReader.Pending(ctx); err != nil { + log.Errorw("Failed to get pending messages", zap.Error(err)) + } else { + if pending != isb.PendingNotAvailable { + ts := timestampedPending{pending: pending, timestamp: time.Now().Unix()} + ms.partitionPendingInfo[partitionName].Append(ts) + } + } + } + } + } +} + +// Expose pending metrics +func (ms *metricsServer) exposePendingMetrics(ctx context.Context) { + if ms.lagReaders == nil { + return + } + lookbackSecondsMap := map[string]int64{"default": ms.lookbackSeconds} // Metrics for autoscaling use key "default" + for k, v := range fixedLookbackSeconds { + lookbackSecondsMap[k] = v + } + ticker := time.NewTicker(ms.refreshInterval) + defer ticker.Stop() + for { + select { + case <-ticker.C: + if ms.lagReaders != nil { + for partitionName := range ms.lagReaders { + for n, i := range lookbackSecondsMap { + if p := ms.calculatePending(i, partitionName); p != isb.PendingNotAvailable { + pending.WithLabelValues(ms.vertex.Spec.PipelineName, ms.vertex.Spec.Name, n, partitionName).Set(float64(p)) + } + } + } + } + case <-ctx.Done(): + return + } + } +} + +// Calculate the avg pending of last seconds +func (ms *metricsServer) calculatePending(seconds int64, partitionName string) int64 { + result := isb.PendingNotAvailable + items := ms.partitionPendingInfo[partitionName].Items() + total := int64(0) + num := int64(0) + now := time.Now().Unix() + for i := len(items) - 1; i >= 0; i-- { + if now-items[i].timestamp < seconds { + total += items[i].pending + num++ + } else { + break + } + } + if num > 0 { + result = total / num + } + return result +} + +// Start function starts the HTTPS service to expose metrics, it returns a shutdown function and an error if any +func (ms *metricsServer) Start(ctx context.Context) (func(ctx context.Context) error, error) { + log := logging.FromContext(ctx) + log.Info("Generating self-signed certificate") + cer, err := sharedtls.GenerateX509KeyPair() + if err != nil { + return nil, fmt.Errorf("failed to generate cert: %w", err) + } + mux := http.NewServeMux() + mux.Handle("/metrics", promhttp.Handler()) + mux.HandleFunc("/readyz", func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusNoContent) + }) + mux.HandleFunc("/livez", func(w http.ResponseWriter, r *http.Request) { + w.WriteHeader(http.StatusNoContent) + }) + mux.HandleFunc("/sidecar-livez", func(w http.ResponseWriter, r *http.Request) { + for _, ex := range ms.healthCheckExecutors { + if err := ex(); err != nil { + log.Errorw("Failed to execute sidecar health check", zap.Error(err)) + w.WriteHeader(http.StatusInternalServerError) + _, _ = w.Write([]byte(err.Error())) + return + } + } + w.WriteHeader(http.StatusNoContent) + }) + 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) + mux.HandleFunc("/debug/pprof/symbol", pprof.Symbol) + mux.HandleFunc("/debug/pprof/trace", pprof.Trace) + } else { + log.Info("Not enabling pprof debug endpoints") + } + + httpServer := &http.Server{ + Addr: fmt.Sprintf(":%d", dfv1.VertexMetricsPort), + Handler: mux, + TLSConfig: &tls.Config{Certificates: []tls.Certificate{*cer}, MinVersion: tls.VersionTLS12}, + } + // Buildup pending information + go ms.buildupPendingInfo(ctx) + // Expose pending metrics + go ms.exposePendingMetrics(ctx) + go func() { + log.Info("Starting metrics HTTPS server") + if err := httpServer.ListenAndServeTLS("", ""); err != nil && err != http.ErrServerClosed { + log.Fatalw("Failed to listen-and-server on HTTPS", zap.Error(err)) + } + log.Info("Metrics server shutdown") + }() + return httpServer.Shutdown, nil +} diff --git a/pkg/metrics/metrics_test.go b/pkg/metrics/metrics_server_test.go similarity index 100% rename from pkg/metrics/metrics_test.go rename to pkg/metrics/metrics_server_test.go diff --git a/pkg/reduce/data_forward.go b/pkg/reduce/data_forward.go index 0d5f521761..e208781edb 100644 --- a/pkg/reduce/data_forward.go +++ b/pkg/reduce/data_forward.go @@ -185,9 +185,10 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { totalBytes := 0 if err != nil { df.log.Errorw("Failed to read from isb", zap.Error(err)) - readMessagesError.With(map[string]string{ + metrics.ReadMessagesError.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Inc() } @@ -257,9 +258,10 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { m.Watermark = time.Time(processorWM) totalBytes += len(m.Payload) } - readBytesCount.With(map[string]string{ + metrics.ReadBytesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName(), }).Add(float64(totalBytes)) @@ -317,15 +319,17 @@ func (df *DataForward) Process(ctx context.Context, messages []*isb.ReadMessage) ctrlMessages = append(ctrlMessages, message) } } - readMessagesCount.With(map[string]string{ + metrics.ReadDataMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName(), }).Add(float64(len(dataMessages))) - totalMessagesCount.With(map[string]string{ + metrics.ReadMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName(), }).Add(float64(len(messages))) @@ -400,11 +404,11 @@ messagesLoop: continue } else if message.EventTime.Before(nextWinAsSeenByWriter.StartTime()) { // if the message doesn't fall in the next window that is about to be closed drop it. df.log.Warnw("Dropping the late message", zap.Time("eventTime", message.EventTime), zap.Time("watermark", message.Watermark), zap.Time("nextWindowToBeClosed", nextWinAsSeenByWriter.StartTime())) - droppedMessagesCount.With(map[string]string{ + metrics.ReduceDroppedMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - LabelReason: "late"}).Inc() + metrics.LabelReason: "late"}).Inc() // mark it as a successfully written message as the message will be acked to avoid subsequent retries writtenMessages = append(writtenMessages, message) @@ -428,11 +432,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.ReduceDroppedMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - LabelReason: "watermark_issue"}).Inc() + metrics.LabelReason: "watermark_issue"}).Inc() continue } @@ -464,12 +468,13 @@ messagesLoop: // writeToPBQ writes to the PBQ. It will return error only if it is not failing to write to PBQ and is in a continuous // error loop, and we have received ctx.Done() via SIGTERM. func (df *DataForward) writeToPBQ(ctx context.Context, m *isb.ReadMessage, p partition.ID, kw window.AlignedKeyedWindower) error { - startTime := time.Now() - defer pbqWriteTime.With(map[string]string{ - metrics.LabelVertex: df.vertexName, - metrics.LabelPipeline: df.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), - }).Observe(float64(time.Since(startTime).Milliseconds())) + defer func(t time.Time) { + metrics.PBQWriteTime.With(map[string]string{ + metrics.LabelVertex: df.vertexName, + metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), + }).Observe(float64(time.Since(t).Milliseconds())) + }(time.Now()) var pbqWriteBackoff = wait.Backoff{ Steps: math.MaxInt, @@ -483,8 +488,8 @@ func (df *DataForward) writeToPBQ(ctx context.Context, m *isb.ReadMessage, p par err := wait.ExponentialBackoff(pbqWriteBackoff, func() (done bool, err error) { rErr := q.Write(context.Background(), m) if rErr != nil { - df.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{ + df.log.Errorw("Failed to write message", zap.String("msgOffSet", m.ReadOffset.String()), zap.String("partitionID", p.String()), zap.Error(rErr)) + metrics.PBQWriteErrorCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), @@ -501,7 +506,7 @@ func (df *DataForward) writeToPBQ(ctx context.Context, m *isb.ReadMessage, p par } // happy path - pbqWriteMessagesCount.With(map[string]string{ + metrics.PBQWriteMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), @@ -534,9 +539,10 @@ func (df *DataForward) ackMessages(ctx context.Context, messages []*isb.ReadMess rErr := o.AckIt() attempt += 1 if rErr != nil { - ackMessageError.With(map[string]string{ + metrics.AckMessageError.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName(), }).Inc() @@ -553,9 +559,10 @@ func (df *DataForward) ackMessages(ctx context.Context, messages []*isb.ReadMess } } df.log.Debugw("Successfully acked message", zap.String("msgOffSet", o.String())) - ackMessagesCount.With(map[string]string{ + metrics.AckMessagesCount.With(map[string]string{ metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName(), }).Inc() diff --git a/pkg/reduce/metrics.go b/pkg/reduce/metrics.go deleted file mode 100644 index b566b5030f..0000000000 --- a/pkg/reduce/metrics.go +++ /dev/null @@ -1,99 +0,0 @@ -/* -Copyright 2022 The Numaproj Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package reduce - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/numaproj/numaflow/pkg/metrics" -) - -const ( - LabelReason = "reason" -) - -// droppedMessagesCount is used to indicate the number of messages dropped -var droppedMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_data_forward", - Name: "dropped_total", - Help: "Total number of Messages Dropped", -}, []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, 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, metrics.LabelVertexReplicaIndex}) - -// pbqWriteTime pbq write latency -var pbqWriteTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "reduce_pbq", - Name: "write_time", - Help: "Entry write time (1 to 5000 microseconds)", - Buckets: prometheus.ExponentialBucketsRange(1, 5000, 5), -}, []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, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// 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, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// totalMessagesCount is used to indicate the number of total messages read -var totalMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_isb_reader", - Name: "total_read", - Help: "Total number of Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// readMessagesCount is used to indicate the number of data messages read -var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_isb_reader", - Name: "data_read", - Help: "Total number of Data Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// 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{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) diff --git a/pkg/reduce/pnf/metrics.go b/pkg/reduce/pnf/metrics.go deleted file mode 100644 index faa2b9c040..0000000000 --- a/pkg/reduce/pnf/metrics.go +++ /dev/null @@ -1,96 +0,0 @@ -/* -Copyright 2022 The Numaproj Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package pnf - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/numaproj/numaflow/pkg/metrics" -) - -// writeMessagesCount is used to indicate the number of messages written -var writeMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_isb_writer", - Name: "write_total", - Help: "Total number of Messages Written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// 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, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// 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, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// dropMessagesCount is used to indicate the number of messages dropped -var dropMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_isb_writer", - Name: "drop_total", - Help: "Total number of Messages Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// dropBytesCount is used to indicate the number of bytes dropped -var dropBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_isb_writer", - Name: "drop_bytes_total", - Help: "Total number of Bytes Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex, metrics.LabelPartitionName}) - -// platformError is used to indicate the number of Internal/Platform errors -var platformError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "reduce_pnf", - Name: "platform_error_total", - Help: "Total number of platform Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) - -// reduceProcessTime reduce ForwardTask processing latency -var reduceProcessTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "reduce_pnf", - Name: "process_time", - Help: "Reduce process time (1 to 1200000 milliseconds)", - Buckets: prometheus.ExponentialBucketsRange(1, 1200000, 5), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelVertexReplicaIndex}) - -// reduceForwardTime is used to indicate the time it took to forward the writeMessages -var reduceForwardTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "reduce_pnf", - Name: "forward_time", - Help: "Reduce forward time (1 to 100000 microseconds)", - Buckets: prometheus.ExponentialBucketsRange(1, 100000, 5), -}, []string{metrics.LabelPipeline, metrics.LabelVertex, metrics.LabelVertexReplicaIndex}) - -// partitionsInFlight is used to indicate the partitions in flight -var partitionsInFlight = promauto.NewGaugeVec(prometheus.GaugeOpts{ - Subsystem: "reduce_pnf", - Name: "partitions_inflight", - Help: "Total number of partitions in flight", -}, []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, metrics.LabelVertexReplicaIndex}) diff --git a/pkg/reduce/pnf/ordered.go b/pkg/reduce/pnf/ordered.go index 6330b34215..51f7098f44 100644 --- a/pkg/reduce/pnf/ordered.go +++ b/pkg/reduce/pnf/ordered.go @@ -116,7 +116,7 @@ func (op *OrderedProcessor) SchedulePnF( doneCh: doneCh, pf: pf, } - partitionsInFlight.With(map[string]string{ + metrics.ReducePartitionsInFlight.With(map[string]string{ metrics.LabelVertex: op.vertexName, metrics.LabelPipeline: op.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(op.vertexReplica)), @@ -134,9 +134,10 @@ func (op *OrderedProcessor) reduceOp(ctx context.Context, t *ForwardTask) { err := t.pf.Process(ctx) if err != nil { if errors.Is(err, ctx.Err()) { - udfError.With(map[string]string{ + metrics.UDFError.With(map[string]string{ metrics.LabelVertex: op.vertexName, metrics.LabelPipeline: op.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(op.vertexReplica)), }).Inc() op.log.Infow("ReduceOp exiting", zap.String("partitionID", t.pf.PartitionID.String()), zap.Error(ctx.Err())) @@ -213,7 +214,7 @@ outerLoop: rm := currElement currElement = currElement.Next() op.taskQueue.Remove(rm) - partitionsInFlight.With(map[string]string{ + metrics.ReducePartitionsInFlight.With(map[string]string{ metrics.LabelVertex: op.vertexName, metrics.LabelPipeline: op.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(op.vertexReplica)), diff --git a/pkg/reduce/pnf/ordered_test.go b/pkg/reduce/pnf/ordered_test.go index 965ae64789..1656029034 100644 --- a/pkg/reduce/pnf/ordered_test.go +++ b/pkg/reduce/pnf/ordered_test.go @@ -23,11 +23,8 @@ import ( "github.com/stretchr/testify/assert" - "github.com/numaproj/numaflow/pkg/forward" - "github.com/numaproj/numaflow/pkg/watermark/wmb" - "github.com/numaproj/numaflow/pkg/window/keyed" - dfv1 "github.com/numaproj/numaflow/pkg/apis/numaflow/v1alpha1" + "github.com/numaproj/numaflow/pkg/forward" "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/isb/testutils" @@ -36,6 +33,8 @@ import ( "github.com/numaproj/numaflow/pkg/reduce/pbq/partition" "github.com/numaproj/numaflow/pkg/reduce/pbq/store/memory" "github.com/numaproj/numaflow/pkg/watermark/generic" + "github.com/numaproj/numaflow/pkg/watermark/wmb" + "github.com/numaproj/numaflow/pkg/window/keyed" ) type myForwardTest struct { diff --git a/pkg/reduce/pnf/processandforward.go b/pkg/reduce/pnf/processandforward.go index 7c4c247e1a..ea91adc165 100644 --- a/pkg/reduce/pnf/processandforward.go +++ b/pkg/reduce/pnf/processandforward.go @@ -92,12 +92,13 @@ func newProcessAndForward(ctx context.Context, // Process method reads messages from the supplied PBQ, invokes UDF to reduce the writeMessages. func (p *processAndForward) Process(ctx context.Context) error { var err error - startTime := time.Now() - 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())) + defer func(t time.Time) { + metrics.ReduceProcessTime.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Observe(float64(time.Since(t).Milliseconds())) + }(time.Now()) // blocking call, only returns the writeMessages after it has read all the messages from pbq p.writeMessages, err = p.UDF.ApplyReduce(ctx, &p.PartitionID, p.pbqReader.ReadCh()) @@ -107,12 +108,13 @@ func (p *processAndForward) Process(ctx context.Context) error { // Forward writes messages to the ISBs, publishes watermark, and invokes GC on PBQ. 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{ - metrics.LabelVertex: p.vertexName, - metrics.LabelPipeline: p.pipelineName, - metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), - }).Observe(float64(time.Since(startTime).Microseconds())) + defer func(t time.Time) { + metrics.ReduceForwardTime.With(map[string]string{ + metrics.LabelVertex: p.vertexName, + metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), + }).Observe(float64(time.Since(t).Microseconds())) + }(time.Now()) // millisecond is the lowest granularity currently supported. processorWM := wmb.Watermark(p.PartitionID.End.Add(-1 * time.Millisecond)) @@ -177,9 +179,10 @@ func (p *processAndForward) whereToStep() map[string][][]isb.Message { for _, msg := range p.writeMessages { to, err = p.whereToDecider.WhereTo(msg.Keys, msg.Tags) if err != nil { - platformError.With(map[string]string{ + metrics.PlatformError.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), }).Inc() p.log.Errorw("Got an error while invoking WhereTo, dropping the message", zap.Strings("keys", msg.Keys), zap.Error(err), zap.Any("partitionID", p.PartitionID)) @@ -242,9 +245,10 @@ func (p *processAndForward) writeToBuffer(ctx context.Context, edgeName 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{ + metrics.WriteMessagesError.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), metrics.LabelPartitionName: p.toBuffers[edgeName][partition].GetName()}).Add(float64(len(failedMessages))) return false, nil @@ -252,27 +256,31 @@ func (p *processAndForward) writeToBuffer(ctx context.Context, edgeName string, return true, nil }) - dropMessagesCount.With(map[string]string{ + metrics.DropMessagesCount.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), metrics.LabelPartitionName: p.toBuffers[edgeName][partition].GetName()}).Add(float64(len(resultMessages) - writeCount)) - dropBytesCount.With(map[string]string{ + metrics.DropBytesCount.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), metrics.LabelPartitionName: p.toBuffers[edgeName][partition].GetName()}).Add(dropBytes) - writeMessagesCount.With(map[string]string{ + metrics.WriteMessagesCount.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), metrics.LabelPartitionName: p.toBuffers[edgeName][partition].GetName()}).Add(float64(writeCount)) - writeBytesCount.With(map[string]string{ + metrics.WriteBytesCount.With(map[string]string{ metrics.LabelVertex: p.vertexName, metrics.LabelPipeline: p.pipelineName, + metrics.LabelVertexType: string(dfv1.VertexTypeReduceUDF), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(p.vertexReplica)), metrics.LabelPartitionName: p.toBuffers[edgeName][partition].GetName()}).Add(writeBytes) return offsets, ctxClosedErr diff --git a/pkg/sinks/blackhole/blackhole.go b/pkg/sinks/blackhole/blackhole.go index 89508d1b97..e72c9e055b 100644 --- a/pkg/sinks/blackhole/blackhole.go +++ b/pkg/sinks/blackhole/blackhole.go @@ -49,7 +49,7 @@ func WithLogger(log *zap.SugaredLogger) Option { } // NewBlackhole returns Blackhole type. -func NewBlackhole(vertex *dfv1.Vertex, +func NewBlackhole(vertexInstance *dfv1.VertexInstance, fromBuffer isb.BufferReader, fetchWatermark fetch.Fetcher, publishWatermark publish.Publisher, @@ -57,9 +57,9 @@ func NewBlackhole(vertex *dfv1.Vertex, opts ...Option) (*Blackhole, error) { bh := new(Blackhole) - name := vertex.Spec.Name + name := vertexInstance.Vertex.Spec.Name bh.name = name - bh.pipelineName = vertex.Spec.PipelineName + bh.pipelineName = vertexInstance.Vertex.Spec.PipelineName for _, o := range opts { if err := o(bh); err != nil { @@ -71,13 +71,13 @@ func NewBlackhole(vertex *dfv1.Vertex, } forwardOpts := []sinkforward.Option{sinkforward.WithLogger(bh.logger)} - if x := vertex.Spec.Limits; x != nil { + if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { forwardOpts = append(forwardOpts, sinkforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := sinkforward.NewDataForward(vertex, fromBuffer, bh, fetchWatermark, publishWatermark, idleManager, forwardOpts...) + isdf, err := sinkforward.NewDataForward(vertexInstance, fromBuffer, bh, fetchWatermark, publishWatermark, idleManager, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/blackhole/blackhole_test.go b/pkg/sinks/blackhole/blackhole_test.go index 0432711d44..07adf80d9d 100644 --- a/pkg/sinks/blackhole/blackhole_test.go +++ b/pkg/sinks/blackhole/blackhole_test.go @@ -46,8 +46,12 @@ func TestBlackhole_Start(t *testing.T) { }, }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{vertex.Spec.Name}) - s, err := NewBlackhole(vertex, fromStep, fetchWatermark, publishWatermark[vertex.Spec.Name], wmb.NewIdleManager(1)) + s, err := NewBlackhole(vertexInstance, fromStep, fetchWatermark, publishWatermark[vertex.Spec.Name], wmb.NewIdleManager(1)) assert.NoError(t, err) stopped := s.Start() diff --git a/pkg/sinks/forward/forward.go b/pkg/sinks/forward/forward.go index cd8af191e7..ad4cf643d0 100644 --- a/pkg/sinks/forward/forward.go +++ b/pkg/sinks/forward/forward.go @@ -21,6 +21,7 @@ import ( "errors" "fmt" "math" + "strconv" "sync" "time" @@ -50,6 +51,7 @@ type DataForward struct { opts options vertexName string pipelineName string + vertexReplica int32 // idleManager manages the idle watermark status. idleManager wmb.IdleManager // wmbChecker checks if the idle watermark is valid. @@ -59,7 +61,7 @@ type DataForward struct { // NewDataForward creates a sink data forwarder. func NewDataForward( - vertex *dfv1.Vertex, + vertexInstance *dfv1.VertexInstance, fromStep isb.BufferReader, toStep isb.BufferWriter, fetchWatermark fetch.Fetcher, @@ -84,10 +86,11 @@ func NewDataForward( wmFetcher: fetchWatermark, wmPublisher: publishWatermark, // should we do a check here for the values not being null? - vertexName: vertex.Spec.Name, - pipelineName: vertex.Spec.PipelineName, - idleManager: idleManager, - wmbChecker: wmb.NewWMBChecker(2), // TODO: make configurable + vertexName: vertexInstance.Vertex.Spec.Name, + pipelineName: vertexInstance.Vertex.Spec.PipelineName, + vertexReplica: vertexInstance.Replica, + idleManager: idleManager, + wmbChecker: wmb.NewWMBChecker(2), // TODO: make configurable Shutdown: Shutdown{ rwlock: new(sync.RWMutex), }, @@ -171,7 +174,7 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { df.opts.logger.Debugw("Read from buffer", zap.String("bufferFrom", df.fromBufferPartition.GetName()), zap.Int64("length", int64(len(readMessages)))) if err != nil { df.opts.logger.Warnw("failed to read fromBufferPartition", zap.Error(err)) - readMessagesError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Inc() + metrics.ReadMessagesError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Inc() } // process only if we have any read messages. There is a natural looping here if there is an internal error while @@ -209,8 +212,8 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { dataMessages = append(dataMessages, m) } } - readMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(dataMessages))) - totalMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readMessages))) + metrics.ReadDataMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(dataMessages))) + metrics.ReadMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readMessages))) // fetch watermark if available // TODO: make it async (concurrent and wait later) @@ -241,7 +244,7 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // only send the data messages for idx, m := range dataMessages { // emit message size metric - readBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(m.Payload))) + metrics.ReadBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(m.Payload))) // assign watermark to the message m.Watermark = time.Time(processorWM) // send message to the channel @@ -288,13 +291,13 @@ func (df *DataForward) forwardAChunk(ctx context.Context) { // implicit return for posterity :-) if err != nil { df.opts.logger.Errorw("Failed to ack from buffer", zap.Error(err)) - ackMessageError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessageError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) return } - ackMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Add(float64(len(readOffsets))) // ProcessingTimes of the entire forwardAChunk - forwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: df.fromBufferPartition.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.ForwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } // ackFromBuffer acknowledges an array of offsets back to fromBufferPartition and is a blocking call or until shutdown has been initiated. @@ -372,10 +375,10 @@ func (df *DataForward) writeToBuffer(ctx context.Context, toBufferPartition isb. needRetry = true // we retry only failed messages failedMessages = append(failedMessages, msg) - writeMessagesError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() + metrics.WriteMessagesError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() // a shutdown can break the blocking loop caused due to InternalErr if ok, _ := df.IsShuttingDown(); ok { - platformError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica))}).Inc() return writeOffsets, fmt.Errorf("writeToBuffer failed, Stop called while stuck on an internal error with failed messages:%d, %v", len(failedMessages), errs) } } @@ -405,10 +408,10 @@ func (df *DataForward) writeToBuffer(ctx context.Context, toBufferPartition isb. } } - dropMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) - dropBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) - writeMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) - writeBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) + metrics.DropMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) + metrics.DropBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) + metrics.WriteMessagesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) + metrics.WriteBytesCount.With(map[string]string{metrics.LabelVertex: df.vertexName, metrics.LabelPipeline: df.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSink), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(df.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) return writeOffsets, nil } diff --git a/pkg/sinks/forward/forward_test.go b/pkg/sinks/forward/forward_test.go index acb209f9f1..84ebe9a257 100644 --- a/pkg/sinks/forward/forward_test.go +++ b/pkg/sinks/forward/forward_test.go @@ -32,6 +32,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" "github.com/numaproj/numaflow/pkg/watermark/generic" "github.com/numaproj/numaflow/pkg/watermark/publish" @@ -125,7 +126,7 @@ func (t *testIdleForwardFetcher) ComputeHeadIdleWMB(int32) wmb.WMB { func TestNewDataForward(t *testing.T) { var ( - testName = "sink_forward" + testName = "forward" batchSize int64 = 10 ) @@ -146,6 +147,11 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -153,7 +159,7 @@ func TestNewDataForward(t *testing.T) { _, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) fetchWatermark := &testForwardFetcher{} - f, err := NewDataForward(vertex, fromStep, to1, fetchWatermark, publishWatermark[testVertexName], wmb.NewIdleManager(1)) + f, err := NewDataForward(vertexInstance, fromStep, to1, fetchWatermark, publishWatermark[testVertexName], wmb.NewIdleManager(1)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -228,11 +234,16 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testIdleForwardFetcher{} publishWatermark := map[string]publish.Publisher{ testVertexName: &testForwarderPublisher{}, } - f, err := NewDataForward(vertex, fromStep, to1, fetchWatermark, publishWatermark[testVertexName], wmb.NewIdleManager(1)) + f, err := NewDataForward(vertexInstance, fromStep, to1, fetchWatermark, publishWatermark[testVertexName], wmb.NewIdleManager(1)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -306,11 +317,16 @@ func TestWriteToBuffer(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} publishWatermark := map[string]publish.Publisher{ "to1": &testForwarderPublisher{}, } - f, err := NewDataForward(vertex, fromStep, buffer, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1)) + f, err := NewDataForward(vertexInstance, fromStep, buffer, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1)) assert.NoError(t, err) assert.False(t, buffer.IsFull()) @@ -348,39 +364,39 @@ func TestWriteToBuffer(t *testing.T) { func validateMetrics(batchSize int64) (err error) { metadata := ` - # HELP sink_forwarder_data_read Total number of Data Messages Read - # TYPE sink_forwarder_data_read counter + # HELP forwarder_data_read_total Total number of Data Messages Read + # TYPE forwarder_data_read_total counter ` expected := ` - sink_forwarder_data_read{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` + forwarder_data_read_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Sink"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` ` - err = testutil.CollectAndCompare(readMessagesCount, strings.NewReader(metadata+expected), "sink_forwarder_data_read") + err = testutil.CollectAndCompare(metrics.ReadDataMessagesCount, strings.NewReader(metadata+expected), "forwarder_data_read_total") if err != nil { return err } writeMetadata := ` - # HELP sink_forwarder_write_total Total number of Messages Written - # TYPE sink_forwarder_write_total counter + # HELP forwarder_write_total Total number of Messages Written + # TYPE forwarder_write_total counter ` writeExpected := ` - sink_forwarder_write_total{partition_name="testVertex",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%d", batchSize) + ` + forwarder_write_total{partition_name="testVertex",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Sink"} ` + fmt.Sprintf("%d", batchSize) + ` ` - err = testutil.CollectAndCompare(writeMessagesCount, strings.NewReader(writeMetadata+writeExpected), "sink_forwarder_write_total") + err = testutil.CollectAndCompare(metrics.WriteMessagesCount, strings.NewReader(writeMetadata+writeExpected), "forwarder_write_total") if err != nil { return err } ackMetadata := ` - # HELP sink_forwarder_ack_total Total number of Messages Acknowledged - # TYPE sink_forwarder_ack_total counter + # HELP forwarder_ack_total Total number of Messages Acknowledged + # TYPE forwarder_ack_total counter ` ackExpected := ` - sink_forwarder_ack_total{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%d", batchSize) + ` + forwarder_ack_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Sink"} ` + fmt.Sprintf("%d", batchSize) + ` ` - err = testutil.CollectAndCompare(ackMessagesCount, strings.NewReader(ackMetadata+ackExpected), "sink_forwarder_ack_total") + err = testutil.CollectAndCompare(metrics.AckMessagesCount, strings.NewReader(ackMetadata+ackExpected), "forwarder_ack_total") if err != nil { return err } @@ -389,7 +405,7 @@ func validateMetrics(batchSize int64) (err error) { } func metricsReset() { - readMessagesCount.Reset() - writeMessagesCount.Reset() - ackMessagesCount.Reset() + metrics.ReadDataMessagesCount.Reset() + metrics.WriteMessagesCount.Reset() + metrics.AckMessagesCount.Reset() } diff --git a/pkg/sinks/forward/metrics.go b/pkg/sinks/forward/metrics.go deleted file mode 100644 index 9409b7f864..0000000000 --- a/pkg/sinks/forward/metrics.go +++ /dev/null @@ -1,116 +0,0 @@ -/* -Copyright 2022 The Numaproj Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package forward - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/numaproj/numaflow/pkg/metrics" -) - -// totalMessagesCount is used to indicate the number of total messages read -var totalMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "total_read", - Help: "Total number of Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readMessagesCount is used to indicate the number of data messages read -var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "data_read", - Help: "Total number of Data Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readBytesCount is to indicate the number of bytes read -var readBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "read_bytes_total", - Help: "Total number of bytes read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readMessagesError is used to indicate the number of errors messages read -var readMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "read_error_total", - Help: "Total number of Read Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeMessagesCount is used to indicate the number of messages written -var writeMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "write_total", - Help: "Total number of Messages Written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeBytesCount is to indicate the number of bytes written -var writeBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "write_bytes_total", - Help: "Total number of bytes written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeMessagesError is used to indicate the number of errors messages written -var writeMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "write_error_total", - Help: "Total number of Write Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropMessagesCount is used to indicate the number of messages dropped -var dropMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "drop_total", - Help: "Total number of Messages Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropBytesCount is to indicate the number of bytes dropped -var dropBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "drop_bytes_total", - Help: "Total number of Bytes Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// ackMessagesCount is used to indicate the number of messages acknowledged -var ackMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "ack_total", - Help: "Total number of Messages Acknowledged", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// ackMessageError is used to indicate the errors in the number of messages acknowledged -var ackMessageError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "ack_error_total", - Help: "Total number of Acknowledged Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// platformError is used to indicate the number of Internal/Platform errors -var platformError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "sink_forwarder", - Name: "platform_error_total", - Help: "Total number of platform Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) - -// forwardAChunkProcessingTime is a histogram to Observe forwardAChunk Processing times as a whole -var forwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "sink_forwarder", - 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, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) diff --git a/pkg/sinks/forward/shutdown_test.go b/pkg/sinks/forward/shutdown_test.go index 3307b8b0eb..4bed71593a 100644 --- a/pkg/sinks/forward/shutdown_test.go +++ b/pkg/sinks/forward/shutdown_test.go @@ -66,8 +66,13 @@ func TestShutDown(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewDataForward(vertex, fromStep, to1, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, to1, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1), WithReadBatchSize(batchSize)) assert.NoError(t, err) stopped := f.Start() // write some data but buffer is not full even though we are not reading @@ -99,8 +104,13 @@ func TestShutDown(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := NewDataForward(vertex, fromStep, to1, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, to1, fetchWatermark, publishWatermark["to1"], wmb.NewIdleManager(1), WithReadBatchSize(batchSize)) assert.NoError(t, err) stopped := f.Start() // write some data such that the fromBufferPartition can be empty, that is toBuffer gets full diff --git a/pkg/sinks/kafka/kafka.go b/pkg/sinks/kafka/kafka.go index 6238c99dc0..ac3bb06ad4 100644 --- a/pkg/sinks/kafka/kafka.go +++ b/pkg/sinks/kafka/kafka.go @@ -57,14 +57,14 @@ func WithLogger(log *zap.SugaredLogger) Option { } // NewToKafka returns ToKafka type. -func NewToKafka(vertex *dfv1.Vertex, +func NewToKafka(vertexInstance *dfv1.VertexInstance, fromBuffer isb.BufferReader, fetchWatermark fetch.Fetcher, publishWatermark publish.Publisher, idleManager wmb.IdleManager, opts ...Option) (*ToKafka, error) { - kafkaSink := vertex.Spec.Sink.Kafka + kafkaSink := vertexInstance.Vertex.Spec.Sink.Kafka toKafka := new(ToKafka) // apply options for kafka sink for _, o := range opts { @@ -78,19 +78,19 @@ func NewToKafka(vertex *dfv1.Vertex, toKafka.log = logging.NewLogger() } toKafka.log = toKafka.log.With("sinkType", "kafka").With("topic", kafkaSink.Topic) - toKafka.name = vertex.Spec.Name - toKafka.pipelineName = vertex.Spec.PipelineName + toKafka.name = vertexInstance.Vertex.Spec.Name + toKafka.pipelineName = vertexInstance.Vertex.Spec.PipelineName toKafka.topic = kafkaSink.Topic toKafka.kafkaSink = kafkaSink forwardOpts := []sinkforward.Option{sinkforward.WithLogger(toKafka.log)} - if x := vertex.Spec.Limits; x != nil { + if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { forwardOpts = append(forwardOpts, sinkforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - f, err := sinkforward.NewDataForward(vertex, fromBuffer, toKafka, fetchWatermark, publishWatermark, idleManager, forwardOpts...) + f, err := sinkforward.NewDataForward(vertexInstance, fromBuffer, toKafka, fetchWatermark, publishWatermark, idleManager, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/kafka/kafka_test.go b/pkg/sinks/kafka/kafka_test.go index 33711d462b..252eebf24a 100644 --- a/pkg/sinks/kafka/kafka_test.go +++ b/pkg/sinks/kafka/kafka_test.go @@ -46,8 +46,12 @@ func TestWriteSuccessToKafka(t *testing.T) { }, }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{vertex.Spec.Name}) - toKafka.isdf, err = sinkforward.NewDataForward(vertex, fromStep, toKafka, fetchWatermark, publishWatermark["testVertex"], wmb.NewIdleManager(1)) + toKafka.isdf, err = sinkforward.NewDataForward(vertexInstance, fromStep, toKafka, fetchWatermark, publishWatermark["testVertex"], wmb.NewIdleManager(1)) assert.NoError(t, err) toKafka.kafkaSink = vertex.Spec.Sink.Kafka toKafka.name = "Test" @@ -98,9 +102,13 @@ func TestWriteFailureToKafka(t *testing.T) { }, }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } toSteps := map[string][]isb.BufferWriter{vertex.Spec.Name: {toKafka}} fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - toKafka.isdf, err = sinkforward.NewDataForward(vertex, fromStep, toKafka, fetchWatermark, publishWatermark["testVertex"], wmb.NewIdleManager(1)) + toKafka.isdf, err = sinkforward.NewDataForward(vertexInstance, fromStep, toKafka, fetchWatermark, publishWatermark["testVertex"], wmb.NewIdleManager(1)) 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 ba59d4750c..273935847e 100644 --- a/pkg/sinks/logger/log.go +++ b/pkg/sinks/logger/log.go @@ -50,7 +50,7 @@ func WithLogger(log *zap.SugaredLogger) Option { } // NewToLog returns ToLog type. -func NewToLog(vertex *dfv1.Vertex, +func NewToLog(vertexInstance *dfv1.VertexInstance, fromBuffer isb.BufferReader, fetchWatermark fetch.Fetcher, publishWatermark publish.Publisher, @@ -58,9 +58,9 @@ func NewToLog(vertex *dfv1.Vertex, opts ...Option) (*ToLog, error) { toLog := new(ToLog) - name := vertex.Spec.Name + name := vertexInstance.Vertex.Spec.Name toLog.name = name - toLog.pipelineName = vertex.Spec.PipelineName + toLog.pipelineName = vertexInstance.Vertex.Spec.PipelineName // use opts in future for specifying logger format etc for _, o := range opts { if err := o(toLog); err != nil { @@ -72,13 +72,13 @@ func NewToLog(vertex *dfv1.Vertex, } forwardOpts := []sinkforward.Option{sinkforward.WithLogger(toLog.logger)} - if x := vertex.Spec.Limits; x != nil { + if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { forwardOpts = append(forwardOpts, sinkforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - isdf, err := sinkforward.NewDataForward(vertex, fromBuffer, toLog, fetchWatermark, publishWatermark, idleManager, forwardOpts...) + isdf, err := sinkforward.NewDataForward(vertexInstance, fromBuffer, toLog, fetchWatermark, publishWatermark, idleManager, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sinks/logger/log_test.go b/pkg/sinks/logger/log_test.go index 3c4a3533f8..7f5d7e8280 100644 --- a/pkg/sinks/logger/log_test.go +++ b/pkg/sinks/logger/log_test.go @@ -52,8 +52,12 @@ func TestToLog_Start(t *testing.T) { }, }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{vertex.Spec.Name}) - s, err := NewToLog(vertex, fromStep, fetchWatermark, publishWatermark[vertex.Spec.Name], wmb.NewIdleManager(1)) + s, err := NewToLog(vertexInstance, fromStep, fetchWatermark, publishWatermark[vertex.Spec.Name], wmb.NewIdleManager(1)) assert.NoError(t, err) stopped := s.Start() @@ -102,9 +106,13 @@ func TestToLog_Forward(t *testing.T) { }, }, }} + vertexInstance1 := &dfv1.VertexInstance{ + Vertex: vertex1, + Replica: 0, + } fetchWatermark1, publishWatermark1 := generic.BuildNoOpWatermarkProgressorsFromBufferList([]string{vertex1.Spec.Name}) - logger1, _ := NewToLog(vertex1, to1, fetchWatermark1, publishWatermark1[vertex1.Spec.Name], wmb.NewIdleManager(1)) + logger1, _ := NewToLog(vertexInstance1, to1, fetchWatermark1, publishWatermark1[vertex1.Spec.Name], wmb.NewIdleManager(1)) logger1Stopped := logger1.Start() toSteps := map[string][]isb.BufferWriter{ @@ -114,7 +122,7 @@ func TestToLog_Forward(t *testing.T) { writeMessages := testutils.BuildTestWriteMessages(int64(20), testStartTime) fetchWatermark, publishWatermark := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) - f, err := sinkforward.NewDataForward(vertex1, fromStep, to1, fetchWatermark, publishWatermark["sinks.logger1"], wmb.NewIdleManager(1), sinkforward.WithReadBatchSize(batchSize)) + f, err := sinkforward.NewDataForward(vertexInstance1, fromStep, to1, fetchWatermark, publishWatermark["sinks.logger1"], wmb.NewIdleManager(1), sinkforward.WithReadBatchSize(batchSize)) assert.NoError(t, err) stopped := f.Start() diff --git a/pkg/sinks/sink.go b/pkg/sinks/sink.go index 7de4be1b26..2fff6c890c 100644 --- a/pkg/sinks/sink.go +++ b/pkg/sinks/sink.go @@ -233,14 +233,14 @@ func (u *SinkProcessor) Start(ctx context.Context) error { func (u *SinkProcessor) getSinker(reader isb.BufferReader, logger *zap.SugaredLogger, fetchWM fetch.Fetcher, publishWM publish.Publisher, idleManager wmb.IdleManager, sinkHandler udsink.SinkApplier) (Sinker, error) { sink := u.VertexInstance.Vertex.Spec.Sink if x := sink.Log; x != nil { - return logsink.NewToLog(u.VertexInstance.Vertex, reader, fetchWM, publishWM, idleManager, logsink.WithLogger(logger)) + return logsink.NewToLog(u.VertexInstance, reader, fetchWM, publishWM, idleManager, logsink.WithLogger(logger)) } else if x := sink.Kafka; x != nil { - return kafkasink.NewToKafka(u.VertexInstance.Vertex, reader, fetchWM, publishWM, idleManager, kafkasink.WithLogger(logger)) + return kafkasink.NewToKafka(u.VertexInstance, reader, fetchWM, publishWM, idleManager, kafkasink.WithLogger(logger)) } else if x := sink.Blackhole; x != nil { - return blackhole.NewBlackhole(u.VertexInstance.Vertex, reader, fetchWM, publishWM, idleManager, blackhole.WithLogger(logger)) + return blackhole.NewBlackhole(u.VertexInstance, reader, fetchWM, publishWM, idleManager, blackhole.WithLogger(logger)) } else if x := sink.UDSink; x != nil { // if the sink is a user defined sink, then we need to pass the sinkHandler to it which will be used to invoke the user defined sink - return udsink.NewUserDefinedSink(u.VertexInstance.Vertex, reader, fetchWM, publishWM, idleManager, sinkHandler, udsink.WithLogger(logger)) + return udsink.NewUserDefinedSink(u.VertexInstance, reader, fetchWM, publishWM, idleManager, sinkHandler, udsink.WithLogger(logger)) } return nil, fmt.Errorf("invalid sink spec") } diff --git a/pkg/sinks/udsink/sink.go b/pkg/sinks/udsink/sink.go index cff6782eca..de991edcbd 100644 --- a/pkg/sinks/udsink/sink.go +++ b/pkg/sinks/udsink/sink.go @@ -51,7 +51,7 @@ func WithLogger(log *zap.SugaredLogger) Option { } // NewUserDefinedSink returns genericSink type. -func NewUserDefinedSink(vertex *dfv1.Vertex, +func NewUserDefinedSink(vertexInstance *dfv1.VertexInstance, fromBuffer isb.BufferReader, fetchWatermark fetch.Fetcher, publishWatermark publish.Publisher, @@ -60,9 +60,9 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, opts ...Option) (*UserDefinedSink, error) { s := new(UserDefinedSink) - name := vertex.Spec.Name + name := vertexInstance.Vertex.Spec.Name s.name = name - s.pipelineName = vertex.Spec.PipelineName + s.pipelineName = vertexInstance.Vertex.Spec.PipelineName for _, o := range opts { if err := o(s); err != nil { return nil, err @@ -73,13 +73,13 @@ func NewUserDefinedSink(vertex *dfv1.Vertex, } forwardOpts := []sinkforward.Option{sinkforward.WithLogger(s.logger)} - if x := vertex.Spec.Limits; x != nil { + if x := vertexInstance.Vertex.Spec.Limits; x != nil { if x.ReadBatchSize != nil { forwardOpts = append(forwardOpts, sinkforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } s.udsink = udsink - isdf, err := sinkforward.NewDataForward(vertex, fromBuffer, s, fetchWatermark, publishWatermark, idleManager, forwardOpts...) + isdf, err := sinkforward.NewDataForward(vertexInstance, fromBuffer, s, fetchWatermark, publishWatermark, idleManager, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sources/forward/data_forward.go b/pkg/sources/forward/data_forward.go index 885a5eeb16..b8d372b070 100644 --- a/pkg/sources/forward/data_forward.go +++ b/pkg/sources/forward/data_forward.go @@ -20,6 +20,7 @@ import ( "context" "errors" "fmt" + "strconv" "sync" "time" @@ -61,6 +62,7 @@ type DataForward struct { opts options vertexName string pipelineName string + vertexReplica int32 // idleManager manages the idle watermark status. idleManager wmb.IdleManager Shutdown @@ -68,7 +70,7 @@ type DataForward struct { // NewDataForward creates a source data forwarder func NewDataForward( - vertex *dfv1.Vertex, + vertexInstance *dfv1.VertexInstance, fromStep isb.BufferReader, toSteps map[string][]isb.BufferWriter, toWhichStepDecider forward.ToWhichStepDecider, @@ -103,8 +105,9 @@ func NewDataForward( toVertexWMStores: toVertexWmStores, toVertexWMPublishers: toVertexWMPublishers, srcWMPublisher: srcWMPublisher, - vertexName: vertex.Spec.Name, - pipelineName: vertex.Spec.PipelineName, + vertexName: vertexInstance.Vertex.Spec.Name, + pipelineName: vertexInstance.Vertex.Spec.PipelineName, + vertexReplica: vertexInstance.Replica, idleManager: idleManager, Shutdown: Shutdown{ rwlock: new(sync.RWMutex), @@ -198,7 +201,7 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { readMessages, err := isdf.reader.Read(ctx, isdf.opts.readBatchSize) if err != nil { isdf.opts.logger.Warnw("failed to read from source", zap.Error(err)) - readMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() + metrics.ReadMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() } // Process only if we have any read messages. @@ -206,7 +209,8 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { if len(readMessages) == 0 { return } - readMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readMessages))) + metrics.ReadDataMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readMessages))) + metrics.ReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readMessages))) // store the offsets of the messages we read from source var readOffsets = make([]isb.Offset, len(readMessages)) @@ -248,7 +252,7 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { // send to transformer only the data messages for idx, m := range readMessages { // emit message size metric - readBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(m.Payload))) + metrics.ReadBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(m.Payload))) // assign watermark to the message m.Watermark = time.Time(processorWM) // send transformer processing work to the channel @@ -261,7 +265,7 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { // context.Done() is closed. wg.Wait() isdf.opts.logger.Debugw("concurrent applyTransformer completed", zap.Int("concurrency", isdf.opts.transformerConcurrency), zap.Duration("took", time.Since(concurrentTransformerProcessingStart))) - concurrentTransformerProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Observe(float64(time.Since(concurrentTransformerProcessingStart).Microseconds())) + metrics.SourceTransformerConcurrentProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Observe(float64(time.Since(concurrentTransformerProcessingStart).Microseconds())) // transformer processing is done. // publish source watermark and assign IsLate attribute based on new event time. @@ -296,7 +300,7 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { // Look for errors in transformer processing if we see even 1 error we return. // Handling partial retrying is not worth ATM. if m.transformerError != nil { - transformerError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() + metrics.SourceTransformerError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() isdf.opts.logger.Errorw("failed to apply source transformer", zap.Error(m.transformerError)) return } @@ -377,13 +381,13 @@ func (isdf *DataForward) forwardAChunk(ctx context.Context) { // implicit return for posterity :-) if err != nil { isdf.opts.logger.Errorw("failed to ack from source", zap.Error(err)) - ackMessageError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessageError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readOffsets))) return } - ackMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readOffsets))) + metrics.AckMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(readOffsets))) // ProcessingTimes of the entire forwardAChunk - forwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.ForwardAChunkProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Observe(float64(time.Since(start).Microseconds())) } func (isdf *DataForward) ackFromSource(ctx context.Context, offsets []isb.Offset) error { @@ -441,10 +445,10 @@ func (isdf *DataForward) writeToBuffer(ctx context.Context, toBufferPartition is needRetry = true // we retry only failed messages failedMessages = append(failedMessages, msg) - writeMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() + metrics.WriteMessagesError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Inc() // a shutdown can break the blocking loop caused due to InternalErr if ok, _ := isdf.IsShuttingDown(); ok { - platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return writeOffsets, fmt.Errorf("writeToBuffer failed, Stop called while stuck on an internal error with failed messages:%d, %v", len(failedMessages), errs) } } @@ -474,10 +478,10 @@ func (isdf *DataForward) writeToBuffer(ctx context.Context, toBufferPartition is } } - dropMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) - dropBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) - writeMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) - writeBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) + metrics.DropMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(totalCount - writeCount)) + metrics.DropBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(dropBytes) + metrics.WriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(float64(writeCount)) + metrics.WriteBytesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: toBufferPartition.GetName()}).Add(writeBytes) return writeOffsets, nil } @@ -485,12 +489,12 @@ func (isdf *DataForward) writeToBuffer(ctx context.Context, toBufferPartition is func (isdf *DataForward) concurrentApplyTransformer(ctx context.Context, readMessagePair <-chan *readWriteMessagePair) { for message := range readMessagePair { start := time.Now() - transformerReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() + metrics.SourceTransformerReadMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Inc() writeMessages, err := isdf.applyTransformer(ctx, message.readMessage) - transformerWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(writeMessages))) + metrics.SourceTransformerWriteMessagesCount.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Add(float64(len(writeMessages))) message.writeMessages = append(message.writeMessages, writeMessages...) message.transformerError = err - transformerProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelPartitionName: isdf.reader.GetName()}).Observe(float64(time.Since(start).Microseconds())) + metrics.SourceTransformerProcessingTime.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica)), metrics.LabelPartitionName: isdf.reader.GetName()}).Observe(float64(time.Since(start).Microseconds())) } } @@ -509,7 +513,7 @@ func (isdf *DataForward) applyTransformer(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("Transformer.Apply, Stop called while stuck on an internal error", zap.Error(err)) - platformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return nil, err } continue @@ -535,7 +539,7 @@ func (isdf *DataForward) whereToStep(writeMessage *isb.WriteMessage, messageToSt // 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{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName}).Inc() + metrics.PlatformError.With(map[string]string{metrics.LabelVertex: isdf.vertexName, metrics.LabelPipeline: isdf.pipelineName, metrics.LabelVertexType: string(dfv1.VertexTypeSource), metrics.LabelVertexReplicaIndex: strconv.Itoa(int(isdf.vertexReplica))}).Inc() return err } return err diff --git a/pkg/sources/forward/data_forward_test.go b/pkg/sources/forward/data_forward_test.go index cceca6c267..c100e4d6a1 100644 --- a/pkg/sources/forward/data_forward_test.go +++ b/pkg/sources/forward/data_forward_test.go @@ -34,6 +34,7 @@ import ( "github.com/numaproj/numaflow/pkg/isb" "github.com/numaproj/numaflow/pkg/isb/stores/simplebuffer" "github.com/numaproj/numaflow/pkg/isb/testutils" + "github.com/numaproj/numaflow/pkg/metrics" "github.com/numaproj/numaflow/pkg/shared/logging" udfapplier "github.com/numaproj/numaflow/pkg/udf/rpc" "github.com/numaproj/numaflow/pkg/watermark/generic" @@ -117,6 +118,11 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -124,7 +130,7 @@ func TestNewDataForward(t *testing.T) { fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) noOpStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, noOpStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, noOpStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -216,10 +222,14 @@ func TestNewDataForward(t *testing.T) { Name: "testVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark := &testForwardFetcher{} toVertexStores := buildToVertexWatermarkStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, &myForwardToAllTest{}, &myForwardToAllTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, &myForwardToAllTest{}, &myForwardToAllTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -363,10 +373,15 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} toVertexStores := buildToVertexWatermarkStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myForwardDropTest{}, myForwardDropTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myForwardDropTest{}, myForwardDropTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -522,10 +537,15 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark := &testForwardFetcher{} toVertexStores := buildToVertexWatermarkStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to11.IsFull()) @@ -648,6 +668,11 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -655,7 +680,7 @@ func TestNewDataForward(t *testing.T) { fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myForwardApplyTransformerErrTest{}, myForwardApplyTransformerErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myForwardApplyTransformerErrTest{}, myForwardApplyTransformerErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -692,9 +717,14 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myForwardApplyWhereToErrTest{}, myForwardApplyWhereToErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.True(t, to1.IsEmpty()) @@ -729,9 +759,14 @@ func TestNewDataForward(t *testing.T) { }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } + fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myForwardInternalErrTest{}, myForwardInternalErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myForwardInternalErrTest{}, myForwardInternalErrTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) assert.False(t, to1.IsFull()) @@ -826,6 +861,10 @@ func TestDataForwardSinglePartition(t *testing.T) { Name: "receivingVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -833,7 +872,7 @@ func TestDataForwardSinglePartition(t *testing.T) { fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, mySourceForwardTest{}, mySourceForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) assert.NoError(t, err) assert.False(t, to1.IsFull()) assert.True(t, to1.IsEmpty()) @@ -876,6 +915,10 @@ func TestDataForwardMultiplePartition(t *testing.T) { Name: "receivingVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } ctx, cancel := context.WithTimeout(context.Background(), time.Second*10) defer cancel() @@ -883,7 +926,7 @@ func TestDataForwardMultiplePartition(t *testing.T) { fetchWatermark := &testForwardFetcher{} toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, mySourceForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, &mySourceForwardTestRoundRobin{}, mySourceForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(5)) assert.NoError(t, err) assert.False(t, to11.IsFull()) assert.False(t, to12.IsFull()) @@ -983,10 +1026,14 @@ func TestWriteToBuffer(t *testing.T) { Name: "testVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myForwardTest{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(value.batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myForwardTest{}, myForwardTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(value.batchSize)) assert.NoError(t, err) assert.False(t, buffer.IsFull()) assert.True(t, buffer.IsEmpty()) @@ -1105,49 +1152,49 @@ func (f myForwardApplyTransformerErrTest) ApplyTransform(_ context.Context, _ *i func validateMetrics(batchSize int64) (err error) { metadata := ` - # HELP source_forwarder_read_total Total number of Messages Read - # TYPE source_forwarder_read_total counter + # HELP forwarder_read_total Total number of Messages Read + # TYPE forwarder_read_total counter ` expected := ` - source_forwarder_read_total{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` + forwarder_read_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` ` - err = testutil.CollectAndCompare(readMessagesCount, strings.NewReader(metadata+expected), "source_forwarder_read_total") + err = testutil.CollectAndCompare(metrics.ReadMessagesCount, strings.NewReader(metadata+expected), "forwarder_read_total") if err != nil { return err } writeMetadata := ` - # HELP source_forwarder_write_total Total number of Messages Written - # TYPE source_forwarder_write_total counter + # HELP forwarder_write_total Total number of Messages Written + # TYPE forwarder_write_total counter ` var writeExpected string if batchSize > 1 { writeExpected = ` - source_forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` - source_forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` + forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` + forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%f", float64(batchSize)/2) + ` ` } else { writeExpected = ` - source_forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` - source_forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%f", float64(0)) + ` + forwarder_write_total{partition_name="to1-1",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%f", float64(batchSize)) + ` + forwarder_write_total{partition_name="to1-2",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%f", float64(0)) + ` ` } - err = testutil.CollectAndCompare(writeMessagesCount, strings.NewReader(writeMetadata+writeExpected), "source_forwarder_write_total") + err = testutil.CollectAndCompare(metrics.WriteMessagesCount, strings.NewReader(writeMetadata+writeExpected), "forwarder_write_total") if err != nil { return err } ackMetadata := ` - # HELP source_forwarder_ack_total Total number of Messages Acknowledged - # TYPE source_forwarder_ack_total counter + # HELP forwarder_ack_total Total number of Messages Acknowledged + # TYPE forwarder_ack_total counter ` ackExpected := ` - source_forwarder_ack_total{partition_name="from",pipeline="testPipeline",vertex="testVertex"} ` + fmt.Sprintf("%d", batchSize) + ` + forwarder_ack_total{partition_name="from",pipeline="testPipeline",replica="0",vertex="testVertex",vertex_type="Source"} ` + fmt.Sprintf("%d", batchSize) + ` ` - err = testutil.CollectAndCompare(ackMessagesCount, strings.NewReader(ackMetadata+ackExpected), "source_forwarder_ack_total") + err = testutil.CollectAndCompare(metrics.AckMessagesCount, strings.NewReader(ackMetadata+ackExpected), "forwarder_ack_total") if err != nil { return err } @@ -1156,9 +1203,9 @@ func validateMetrics(batchSize int64) (err error) { } func metricsReset() { - readMessagesCount.Reset() - writeMessagesCount.Reset() - ackMessagesCount.Reset() + metrics.ReadMessagesCount.Reset() + metrics.WriteMessagesCount.Reset() + metrics.AckMessagesCount.Reset() } // buildPublisherMap builds OTStore and publisher for each toBuffer diff --git a/pkg/sources/forward/metrics.go b/pkg/sources/forward/metrics.go deleted file mode 100644 index 8d11f315d2..0000000000 --- a/pkg/sources/forward/metrics.go +++ /dev/null @@ -1,146 +0,0 @@ -/* -Copyright 2022 The Numaproj Authors. - -Licensed under the Apache License, Version 2.0 (the "License"); -you may not use this file except in compliance with the License. -You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - -Unless required by applicable law or agreed to in writing, software -distributed under the License is distributed on an "AS IS" BASIS, -WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -See the License for the specific language governing permissions and -limitations under the License. -*/ - -package forward - -import ( - "github.com/prometheus/client_golang/prometheus" - "github.com/prometheus/client_golang/prometheus/promauto" - - "github.com/numaproj/numaflow/pkg/metrics" -) - -// readMessagesCount is used to indicate the number of messages read -var readMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "read_total", - Help: "Total number of Messages Read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readBytesCount is to indicate the number of bytes read -var readBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "read_bytes_total", - Help: "Total number of bytes read", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// readMessagesError is used to indicate the number of errors messages read -var readMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "read_error_total", - Help: "Total number of Read Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeMessagesCount is used to indicate the number of messages written -var writeMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "write_total", - Help: "Total number of Messages Written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeBytesCount is to indicate the number of bytes written -var writeBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "write_bytes_total", - Help: "Total number of bytes written", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// writeMessagesError is used to indicate the number of errors messages written -var writeMessagesError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "write_error_total", - Help: "Total number of Write Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropMessagesCount is used to indicate the number of messages dropped -var dropMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "drop_total", - Help: "Total number of Messages Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// dropBytesCount is to indicate the number of bytes dropped -var dropBytesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "drop_bytes_total", - Help: "Total number of Bytes Dropped", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// ackMessagesCount is used to indicate the number of messages acknowledged -var ackMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "ack_total", - Help: "Total number of Messages Acknowledged", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// ackMessageError is used to indicate the errors in the number of messages acknowledged -var ackMessageError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "ack_error_total", - Help: "Total number of Acknowledged Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// transformerError is used to indicate the number of source transformer errors -var transformerError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "transformer_error_total", - Help: "Total number of source transformer Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// platformError is used to indicate the number of Internal/Platform errors -var platformError = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "platform_error_total", - Help: "Total number of platform Errors", -}, []string{metrics.LabelVertex, metrics.LabelPipeline}) - -// forwardAChunkProcessingTime is a histogram to Observe forwardAChunk Processing times as a whole -var forwardAChunkProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "source_forwarder", - 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, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// transformerProcessingTime is a histogram to Observe Source Transformer Processing times as a whole -var transformerProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "source_forwarder", - Name: "transformer_processing_time", - Help: "Processing times of source transformer (100 microseconds to 15 minutes)", - Buckets: prometheus.ExponentialBucketsRange(100, 60000000*15, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// concurrentTransformerProcessingTime is a histogram to Observe Source Transformer Processing times as a whole -var concurrentTransformerProcessingTime = promauto.NewHistogramVec(prometheus.HistogramOpts{ - Subsystem: "source_forwarder", - Name: "concurrent_transformer_processing_time", - Help: "Processing times of Concurrent source transformer (100 microseconds to 20 minutes)", - Buckets: prometheus.ExponentialBucketsRange(100, 60000000*20, 10), -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// transformerReadMessagesCount is used to indicate the number of messages read by source transformer -var transformerReadMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "transformer_read_total", - Help: "Total number of Messages Read by source transformer", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) - -// transformerWriteMessagesCount is used to indicate the number of messages written by source transformer -var transformerWriteMessagesCount = promauto.NewCounterVec(prometheus.CounterOpts{ - Subsystem: "source_forwarder", - Name: "transformer_write_total", - Help: "Total number of Messages Written by source transformer", -}, []string{metrics.LabelVertex, metrics.LabelPipeline, metrics.LabelPartitionName}) diff --git a/pkg/sources/forward/shutdown_test.go b/pkg/sources/forward/shutdown_test.go index c7583fd522..4c3375e564 100644 --- a/pkg/sources/forward/shutdown_test.go +++ b/pkg/sources/forward/shutdown_test.go @@ -81,10 +81,14 @@ func TestInterStepDataForward(t *testing.T) { Name: "testVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexWmStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexWmStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexWmStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) stopped := f.Start() // write some data but buffer is not full even though we are not reading @@ -115,10 +119,14 @@ func TestInterStepDataForward(t *testing.T) { Name: "testVertex", }, }} + vertexInstance := &dfv1.VertexInstance{ + Vertex: vertex, + Replica: 0, + } fetchWatermark, _ := generic.BuildNoOpWatermarkProgressorsFromBufferMap(toSteps) toVertexWmStores := buildNoOpToVertexStores(toSteps) - f, err := NewDataForward(vertex, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexWmStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) + f, err := NewDataForward(vertexInstance, fromStep, toSteps, myShutdownTest{}, myShutdownTest{}, fetchWatermark, TestSourceWatermarkPublisher{}, toVertexWmStores, wmb.NewIdleManager(len(toSteps)), WithReadBatchSize(batchSize)) assert.NoError(t, err) stopped := f.Start() // write some data such that the reader can be empty, that is toBuffer gets full diff --git a/pkg/sources/generator/tickgen.go b/pkg/sources/generator/tickgen.go index 9ecb6606ae..f332726571 100644 --- a/pkg/sources/generator/tickgen.go +++ b/pkg/sources/generator/tickgen.go @@ -221,7 +221,7 @@ func NewMemGen( genSrc.sourcePublishWM = genSrc.buildSourceWatermarkPublisher(publishWMStores) // we pass in the context to forwarder as well so that it can shut down when we cancelFn the context - forwarder, err := sourceforward.NewDataForward(vertexInstance.Vertex, genSrc, writers, fsd, transformerApplier, fetchWM, genSrc, toVertexPublisherStores, idleManager, forwardOpts...) + forwarder, err := sourceforward.NewDataForward(vertexInstance, genSrc, writers, fsd, transformerApplier, fetchWM, genSrc, toVertexPublisherStores, idleManager, forwardOpts...) if err != nil { return nil, err } diff --git a/pkg/sources/http/http.go b/pkg/sources/http/http.go index ec01b26456..dfd1ef2cb0 100644 --- a/pkg/sources/http/http.go +++ b/pkg/sources/http/http.go @@ -200,7 +200,7 @@ func New( } } - h.forwarder, err = sourceforward.NewDataForward(vertexInstance.Vertex, h, writers, fsd, transformerApplier, fetchWM, h, toVertexPublisherStores, idleManager, forwardOpts...) + h.forwarder, err = sourceforward.NewDataForward(vertexInstance, h, writers, fsd, transformerApplier, fetchWM, h, toVertexPublisherStores, idleManager, 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 d90537bc82..521d8d5a7e 100644 --- a/pkg/sources/kafka/reader.go +++ b/pkg/sources/kafka/reader.go @@ -401,7 +401,7 @@ func NewKafkaSource( forwardOpts = append(forwardOpts, sourceforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := sourceforward.NewDataForward(vertexInstance.Vertex, kafkaSource, writers, fsd, transformerApplier, fetchWM, kafkaSource, toVertexPublisherStores, idleManager, forwardOpts...) + forwarder, err := sourceforward.NewDataForward(vertexInstance, kafkaSource, writers, fsd, transformerApplier, fetchWM, kafkaSource, toVertexPublisherStores, idleManager, 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 89b4ecb374..2d99574f77 100644 --- a/pkg/sources/nats/nats.go +++ b/pkg/sources/nats/nats.go @@ -91,7 +91,7 @@ func New( forwardOpts = append(forwardOpts, sourceforward.WithReadBatchSize(int64(*x.ReadBatchSize))) } } - forwarder, err := sourceforward.NewDataForward(vertexInstance.Vertex, n, writers, fsd, transformerApplier, fetchWM, n, toVertexPublisherStores, idleManager, forwardOpts...) + forwarder, err := sourceforward.NewDataForward(vertexInstance, n, writers, fsd, transformerApplier, fetchWM, n, toVertexPublisherStores, idleManager, forwardOpts...) if err != nil { n.logger.Errorw("Error instantiating the forwarder", zap.Error(err)) return nil, err diff --git a/pkg/sources/udsource/user_defined_source.go b/pkg/sources/udsource/user_defined_source.go index de910560cb..a73638fd3e 100644 --- a/pkg/sources/udsource/user_defined_source.go +++ b/pkg/sources/udsource/user_defined_source.go @@ -114,7 +114,7 @@ func New( } } var err error - u.forwarder, err = sourceforward.NewDataForward(vertexInstance.Vertex, u, writers, fsd, transformer, fetchWM, u, toVertexPublisherStores, idleManager, forwardOpts...) + u.forwarder, err = sourceforward.NewDataForward(vertexInstance, u, writers, fsd, transformer, fetchWM, u, toVertexPublisherStores, idleManager, forwardOpts...) if err != nil { u.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 ef83663dce..fa40069bcc 100644 --- a/pkg/udf/map_udf.go +++ b/pkg/udf/map_udf.go @@ -228,7 +228,7 @@ func (u *MapUDFProcessor) Start(ctx context.Context) error { } } // create a forwarder for each partition - forwarder, err := forward.NewInterStepDataForward(u.VertexInstance.Vertex, readers[index], writers, conditionalForwarder, mapHandler, mapStreamHandler, fetchWatermark, publishWatermark, idleManager, opts...) + forwarder, err := forward.NewInterStepDataForward(u.VertexInstance, readers[index], writers, conditionalForwarder, mapHandler, mapStreamHandler, fetchWatermark, publishWatermark, idleManager, opts...) if err != nil { return err } diff --git a/test/e2e/functional_test.go b/test/e2e/functional_test.go index 2b2459cf03..fdb40b3b19 100644 --- a/test/e2e/functional_test.go +++ b/test/e2e/functional_test.go @@ -227,7 +227,7 @@ func (s *FunctionalSuite) TestDropOnFull() { time.Sleep(time.Second * 5) w.SendMessageTo(pipelineName, "in", NewHttpPostRequest().WithBody([]byte("2"))) - expectedDropMetric := `forwarder_drop_total{partition_name="numaflow-system-drop-on-full-sink-0",pipeline="drop-on-full",vertex="in"} 1` + expectedDropMetric := `forwarder_drop_total{partition_name="numaflow-system-drop-on-full-sink-0",pipeline="drop-on-full",replica="0",vertex="in",vertex_type="Source"} 1` // wait for the drop metric to be updated, time out after 10s. timeoutChan := time.After(time.Second * 10) ticker := time.NewTicker(time.Second * 2)