hlteoh37 commented on code in PR #3: URL: https://github.com/apache/flink-connector-prometheus/pull/3#discussion_r1829043534
########## docs/content/docs/connectors/datastream/prometheus.md: ########## @@ -0,0 +1,473 @@ +--- +title: Prometheus +weight: 5 +type: docs +aliases: + - /dev/connectors/prometheus.html + - /apis/streaming/connectors/prometheus.html +--- + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Prometheus Sink + +This sink connector can be used to write **data** to Prometheus-compatible storage, using the [Remote Write](https://prometheus.io/docs/specs/remote_write_spec/) Prometheus interface. + +The Prometheus-compatible backend must support [Remote Write 1.0](https://prometheus.io/docs/specs/remote_write_spec/) standard API, and the Remote Write endpoint must be enabled. + +{{< hint warn >}}This connector is not meant for sending internal Flink metrics to Prometheus. +To publish Flink metrics, for monitoring health and operations of the Flink cluster, you should use +[Metric Reporters](../../../deployment/metric_reporters/).{{< /hint >}} + +To use the connector, add the following Maven dependency to your project: + +{{< connector_artifact flink-connector-prometheus prometheus >}} + +## Usage + +The Prometheus sink provides a builder class to build a `PrometheusSink` instance. The code snippets below shows +how to build a `PrometheusSink` with a basic configuration, and an optional [request signer](#request-signer). + +```java +PrometheusSink sink = PrometheusSink.builder() + .setPrometheusRemoteWriteUrl(prometheusRemoteWriteUrl) + .setRequestSigner(new AmazonManagedPrometheusWriteRequestSigner(prometheusRemoteWriteUrl, prometheusRegion)) // Optional + .build(); +``` +The only **required** configuration is `prometheusRemoteWriteUrl`. All other configurations are optional. + +If your sink has parallelism > 1, you need to ensure the stream is keyed using the `PrometheusTimeSeriesLabelsAndMetricNameKeySelector` +key selector, so that all samples of the same time-series are in the same partition and order is not lost. +See [Sink parallelism and keyed streams](#sink-parallelism-and-keyed-streams) for more details. + + +### Input data objects + +The sink expects `PrometheusTimeSeries` records as input. +Your input data must be converted into `PrometheusTimeSeries`, using a map or flatMap operator, before the sending to the sink. + +`PrometheusTimeSeries` instances are immutable and cannot be reused. You can use the [builder](#populating-a-prometheustimeseries) +to create and populate instances. + +A `PrometheusTimeSeries` represents a single time-series record when sent to the Remote Write interface. Each time-series +record may contain multiple samples. + +{{< hint info >}} +In the context of Prometheus, the term "time-series" is overloaded. +It means both *a series of samples with a unique set of labels* (a time-series in the underlying time-series database), +and *a record sent to the Remote Write interface*. A `PrometheusTimeSeries` instance represents a record sent to the interface. + +The two concepts are related, because time-series "records" with the same sets of labels are sent to the same +"database time-series".{{< /hint >}} + +Each `PrometheusTimeSeries` record contains: + +- One **`metricName`**. A string that is translated into the value of the `__name__` label. +- Zero or more **`Label`** entries. Each label has a `key` and a `value`, both `String`. Labels represent additional dimensions of the samples. Duplicate Label keys are not allowed. +- One or more **`Sample`**. Each sample has a `value` (`double`) representing the measure, and a `timestamp` (`long`) representing the time of the measure, in milliseconds from the Epoch. Duplicate timestamps in the same record are not allowed. + +The following pseudocode represents the structure of a `PrometheusTimeSeries` record: + +``` +PrometheusTimeSeries + + --> (1) metricName <String> + + --> (0..*) Label + + name <String> + + value <String> + + --> 1..* Sample + + timestamp <long> + + value <double> +``` Review Comment: The numbers are a little confusing - can we instead add comments on the right to explain optional / non-optional? ########## docs/content/docs/connectors/datastream/prometheus.md: ########## @@ -0,0 +1,473 @@ +--- +title: Prometheus +weight: 5 +type: docs +aliases: + - /dev/connectors/prometheus.html + - /apis/streaming/connectors/prometheus.html +--- + +<!-- +Licensed to the Apache Software Foundation (ASF) under one +or more contributor license agreements. See the NOTICE file +distributed with this work for additional information +regarding copyright ownership. The ASF licenses this file +to you 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. +--> + +# Prometheus Sink + +This sink connector can be used to write **data** to Prometheus-compatible storage, using the [Remote Write](https://prometheus.io/docs/specs/remote_write_spec/) Prometheus interface. + +The Prometheus-compatible backend must support [Remote Write 1.0](https://prometheus.io/docs/specs/remote_write_spec/) standard API, and the Remote Write endpoint must be enabled. + +{{< hint warn >}}This connector is not meant for sending internal Flink metrics to Prometheus. +To publish Flink metrics, for monitoring health and operations of the Flink cluster, you should use +[Metric Reporters](../../../deployment/metric_reporters/).{{< /hint >}} + +To use the connector, add the following Maven dependency to your project: + +{{< connector_artifact flink-connector-prometheus prometheus >}} + +## Usage + +The Prometheus sink provides a builder class to build a `PrometheusSink` instance. The code snippets below shows +how to build a `PrometheusSink` with a basic configuration, and an optional [request signer](#request-signer). + +```java +PrometheusSink sink = PrometheusSink.builder() + .setPrometheusRemoteWriteUrl(prometheusRemoteWriteUrl) + .setRequestSigner(new AmazonManagedPrometheusWriteRequestSigner(prometheusRemoteWriteUrl, prometheusRegion)) // Optional + .build(); +``` +The only **required** configuration is `prometheusRemoteWriteUrl`. All other configurations are optional. + +If your sink has parallelism > 1, you need to ensure the stream is keyed using the `PrometheusTimeSeriesLabelsAndMetricNameKeySelector` +key selector, so that all samples of the same time-series are in the same partition and order is not lost. +See [Sink parallelism and keyed streams](#sink-parallelism-and-keyed-streams) for more details. + + +### Input data objects + +The sink expects `PrometheusTimeSeries` records as input. +Your input data must be converted into `PrometheusTimeSeries`, using a map or flatMap operator, before the sending to the sink. + +`PrometheusTimeSeries` instances are immutable and cannot be reused. You can use the [builder](#populating-a-prometheustimeseries) +to create and populate instances. + +A `PrometheusTimeSeries` represents a single time-series record when sent to the Remote Write interface. Each time-series +record may contain multiple samples. + +{{< hint info >}} +In the context of Prometheus, the term "time-series" is overloaded. +It means both *a series of samples with a unique set of labels* (a time-series in the underlying time-series database), +and *a record sent to the Remote Write interface*. A `PrometheusTimeSeries` instance represents a record sent to the interface. + +The two concepts are related, because time-series "records" with the same sets of labels are sent to the same +"database time-series".{{< /hint >}} + +Each `PrometheusTimeSeries` record contains: + +- One **`metricName`**. A string that is translated into the value of the `__name__` label. +- Zero or more **`Label`** entries. Each label has a `key` and a `value`, both `String`. Labels represent additional dimensions of the samples. Duplicate Label keys are not allowed. +- One or more **`Sample`**. Each sample has a `value` (`double`) representing the measure, and a `timestamp` (`long`) representing the time of the measure, in milliseconds from the Epoch. Duplicate timestamps in the same record are not allowed. + +The following pseudocode represents the structure of a `PrometheusTimeSeries` record: + +``` +PrometheusTimeSeries + + --> (1) metricName <String> + + --> (0..*) Label + + name <String> + + value <String> + + --> 1..* Sample + + timestamp <long> + + value <double> +``` + +{{< hint info >}}The set of Labels and metricName are the unique identifiers of the database time-series. +A composite of all Labels and metricName is also the key you should use to partition data, both inside the Flink application +and upstream, to guarantee ordering per time-series is retained.{{< /hint >}} + + +### Populating a PrometheusTimeSeries + +`PrometheusTimeSeries` provides a builder interface. + +```java +PrometheusTimeSeries inputRecord = + PrometheusTimeSeries.builder() + .withMetricName(metricName) + .addLabel("DeviceID", instanceId) + .addLabel("RoomID", roomId) + .addSample(measurement1, time1) + .addSample(measurement2, time2) + .build(); +``` + +Each `PrometheusTimeSeries` instance can contain multiple samples. Call `.addSample(...)` for each of them. +The order in which samples are added is retained. +The maximum number of samples per record is limited by the `maxBatchSizeInSamples` configuration. + +Aggregating multiple samples into a single `PrometheusTimeSeries` record may improve write performances. + + +## Prometheus remote-write constraints + +Prometheus imposes strict constrains on data format and on ordering. +Any write request containing records that violate these constraints is rejected. + +See [Remote Write specification](https://prometheus.io/docs/specs/remote_write_spec) for details about these constrains. + +In practice, the behavior when writing data to a Prometheus-compatible backend depends on the Prometheus implementation and configuration. +In some cases, these constraints are relaxed, and writes violating the Remote Write specifications may be accepted. + +For this reason, this connector **does not enforce** any data constraints directly. +The user is responsible for sending data to the sink that does not violate the actual constraints of your Prometheus implementation. +See [User responsibilities](#user-responsibilities) for more details. + + +### Ordering constraints + +Remote Write specifications require multiple ordering constraints: + +1. **Labels** within a `PrometheusTimeSeries` record must be in lexicographical **order by `key`**. +2. **Samples** within a `PrometheusTimeSeries` record must be in **`timestamp` order**, from older to newer. +3. **All samples** belonging to the **same time-series** (a unique set of labels and metricName) must be written in **`timestamp` order**. +4. Within the **same time-series**, duplicate samples with the **same timestamp** are not allowed + +When the Prometheus-compatible backend implementation supports [*out-of-order time windows*](https://prometheus.io/docs/prometheus/latest/configuration/configuration/#tsdb) and the option is enabled, sample ordering constraint is relaxed. You can send out of order data within the configured window. + + +### Format constraints + +The `PrometheusTimeSeries` records sent to the sink must also respect the following constraints: + +* **`metricName`** must be defined and non-empty. The connector translates this property into the value of the `__name__` label. +* Label **names** must follow the regex `[a-zA-Z:_]([a-zA-Z0-9_:])`. In particular, labels names containing `@`, `$`, `!`, `.` (dot), or any punctuation marks (except colon `:` and hyphen `-`) are **not valid**. +* Label **names** must not begin with `__` (double underscore). These label names are reserved. +* No duplicate Label **names** are allowed. +* Label **values** and `metricName` may contain any UTF-8 character. +* Label **values** cannot be empty (null or empty string). + +The `PrometheusTimeSeries` builder does not enforce these constraints. + +### User responsibilities + +The user is responsible for sending records to the sink (`PrometheusTimeSeries`) that respect format and ordering +constraints required by your Prometheus implementation. +The connector does not perform any validation or reordering. + +Sample ordering by timestamp is particularly important. +Samples belonging to the same time-series, i.e. with the same set of Labels and the same metric name, must be written in timestamp order. +Source data must be generated in order. The order must also be retained before the sink. When partitioning the data, records with same set of labels and metric name must be sent to the same partition in order to retain ordering. + +Malformed or out of order records written to the Remote Write endpoint are rejected and dropped by the sink. This may cause data loss. + +Any record violating ordering sent to the sink is dropped and may cause other records batched in the same write-request to be dropped. +For more details, see [Connector guarantees](#connector-guarantees). + + +### Sink parallelism and keyed streams + +Each sink operator sub-task uses a single thread to send write requests to the Remote Write endpoint, and `PrometheusTimeSeries` records +are written in the same order as they are received by the sub-task. + +To ensure all records belonging to the same time-series (i.e. `PrometheusTimeSeries` with identical list of `Label` and `metricName`) +are written by the same sink subtask, the stream of `PrometheusTimeSeries` must be keyed using `PrometheusTimeSeriesLabelsAndMetricNameKeySelector`. + +```java +DataStream<MyRecord> inputRecords; +// ... +KeyedStream<PrometheusTimeSeries> timeSeries = inputRecords + .map(new MyRecordToTimeSeriesMapper()) + .keyBy(new PrometheusTimeSeriesLabelsAndMetricNameKeySelector()); + +timeSeries.sinkTo(prometheusSink); +``` + +Using this key selector prevents accidental out-of-orderness due to repartitioning before the sink operator. +However, the user is responsible to retain ordering to this point by partitioning the records correctly. + + +## Error handling + +This paragraph covers handling of errors conditions when writing data to the Remote Write endpoint. + +There are four types of error conditions: + +1. Retryable errors due to temporary error conditions in the Remote-Write server or due to throttling: `5xx` or `429` http responses, connectivity issues. +2. Non-retryable errors due to data violating any of the constraints, malformed data or out-of-order samples: `4xx` http responses, except `429`, `403` and `404`. +3. Fatal error response: authentication failures (`403` http response) or incorrect endpoint path (`404` http response). +4. Any other unexpected failure while writing, due to exceptions while writing to the Prometheus endpoint. + + +### On-error behaviors + +When any of the above error is encountered, the connector implements one of these two behaviors: + +1. `FAIL`: throw an unhandled exception, the job fails +2. `DISCARD_AND_CONTINUE`: discard the request that caused the error, and continue with the next record. + +When a write request is discarded on `DISCARD_AND_CONTINUE`, all the following happens: + +1. Log a message at `WARN` level with the cause of the error. When the error is caused by a response from the endpoint, the payload of the response from the endpoint is included. +2. Increase [counter](#connector-metrics) metrics, to count the number of rejected samples and write requests. +3. **Drop the entire write request**. Note that due to [batching](#batching), a write request may contain multiple `PrometheusTimeSeries`. +4. Continue with the next input record. + + +{{< hint info >}}Prometheus Remote Write does not support partial failures. +Due to [batching](#batching), a single write request may contain multiple input records (`PrometheusTimeSeries`). +If a request contains even a single offending record, the entire write request (the entire batch) must be discarded.{{< /hint >}} + + +#### Retryable error responses + +A typical retryable error condition is enpoint throttling, with a `429, Too Many Requests` response. Review Comment: We might want to use US english! -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
