Skip to content

[FLINK-33139] Add Prometheus Sink Table API #22

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Open
wants to merge 1 commit into
base: main
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
2 changes: 1 addition & 1 deletion .github/PULL_REQUEST_TEMPLATE.md
Original file line number Diff line number Diff line change
Expand Up @@ -32,7 +32,7 @@ This change added tests and can be verified as follows:
*(example:)*
- *Added integration tests for end-to-end deployment*
- *Added unit tests*
- *Manually verified by running the Kinesis connector on a local Flink cluster.*
- *Manually verified by running the Prometheus connector on a local Flink cluster.*

## Significant changes
*(Please check any boxes [x] if the answer is "yes". You can first publish the PR and check them afterwards, for convenience.)*
Expand Down
22 changes: 22 additions & 0 deletions docs/data/prometheus.yml
Original file line number Diff line number Diff line change
@@ -0,0 +1,22 @@
################################################################################
# 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.
################################################################################

version: 1.1-SNAPSHOT
variants:
- maven: flink-connector-prometheus
- maven: flink-connector-prometheus-request-signer-amp
6 changes: 6 additions & 0 deletions flink-connector-prometheus-request-signer-amp/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -54,6 +54,12 @@ under the License.
<scope>provided</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-aws-base</artifactId>
<version>5.0.0-1.20</version>
</dependency>

<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>auth</artifactId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -59,12 +59,19 @@ public class AmazonManagedPrometheusWriteRequestSigner implements PrometheusRequ
* @param awsRegion Region of the AMP workspace
*/
public AmazonManagedPrometheusWriteRequestSigner(String remoteWriteUrl, String awsRegion) {
this(remoteWriteUrl, awsRegion, DefaultCredentialsProvider.create());
}

public AmazonManagedPrometheusWriteRequestSigner(
String remoteWriteUrl, String awsRegion, AwsCredentialsProvider credentialsProvider) {
Preconditions.checkArgument(
StringUtils.isNotBlank(awsRegion), "awsRegion cannot be null or empty");
Preconditions.checkArgument(
StringUtils.isNotBlank(remoteWriteUrl), "remoteWriteUrl cannot be null or empty");

this.awsRegion = awsRegion;
this.credentialsProvider = credentialsProvider;

try {
this.remoteWriteUrl = new URL(remoteWriteUrl);
} catch (MalformedURLException e) {
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,51 @@
/*
* 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.
*/

package org.apache.flink.connector.prometheus.sink.aws;

import org.apache.flink.connector.aws.util.AWSGeneralUtil;
import org.apache.flink.connector.prometheus.sink.PrometheusRequestSigner;
import org.apache.flink.connector.prometheus.table.PrometheusConfig;
import org.apache.flink.connector.prometheus.table.PrometheusDynamicRequestSignerFactory;

import software.amazon.awssdk.auth.credentials.AwsCredentialsProvider;

import java.util.Properties;

public class AmazonManagedPrometheusWriteRequestSignerFactory
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'd be interested to see how this is used to set up the request signer declaratively in SQL

Copy link
Author

@darenwkt darenwkt Jun 2, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Request signer is now a plugin (through Java Service Provider Interface (SPI)), and can be specified by user in the table config (metric.request-signer), for example:

CREATE TABLE PromTable (
  `my_metric_name` BIGINT,
  `my_label_1` BIGINT,
  `my_label_2` BIGINT,
  `sample_value` BIGINT,
  `sample_ts` TIMESTAMP(3)
)
WITH (
  'connector' = 'prometheus',
  'aws.region' = 'us-east-1',
  'metric.request-signer' = 'amazon-managed-prometheus',
  'metric.endpoint-url' = 'https://aps-workspaces.us-east-1.amazonaws.com/workspaces/abc',
  'metric.name' = 'my_metric_name',
  'metric.label.keys' = '[my_label_1,my_label_2]',
  'metric.sample.key' = 'sample_value',
  'metric.sample.timestamp' = 'sample_ts',
  'sink.batch.max-size' = '2',
  'sink.flush-buffer.size' = '1'
);

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

please explain "
'sink.batch.max-size' = '2',
'sink.flush-buffer.size' = '1'
have idea what the intend is, just want to confirm...

Copy link
Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi @georgelza,

'sink.batch.max-size' determines the max size of batch records before flushing it to sink. I would recommend checking the asyncSink FLIP for more details on how it works https://cwiki.apache.org/confluence/display/FLINK/FLIP-171%3A+Async+Sink

Another helpful resource is to check other similar connector and how these config are used https://nightlies.apache.org/flink/flink-docs-master/docs/connectors/table/kinesis/#sink-batch-max-size

When this PR is merged and released, we should have equivalent docs on what each of these config mean as well

implements PrometheusDynamicRequestSignerFactory {
public AmazonManagedPrometheusWriteRequestSignerFactory() {}

@Override
public String requestSignerIdentifer() {
return "amazon-managed-prometheus";
}

@Override
public PrometheusRequestSigner getRequestSigner(PrometheusConfig config) {
Properties properties = config.toProperties();
AWSGeneralUtil.validateAwsConfiguration(properties);

final AwsCredentialsProvider credentialsProvider =
AWSGeneralUtil.getCredentialsProvider(properties);
final String awsRegion = AWSGeneralUtil.getRegion(properties).toString();
final String remoteWriteUrl = config.getRemoteWriteEndpointUrl();

return new AmazonManagedPrometheusWriteRequestSigner(
remoteWriteUrl, awsRegion, credentialsProvider);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,16 @@
# 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.

org.apache.flink.connector.prometheus.sink.aws.AmazonManagedPrometheusWriteRequestSignerFactory
Original file line number Diff line number Diff line change
@@ -0,0 +1,68 @@
package org.apache.flink.connector.prometheus.sink.aws;

import org.apache.flink.configuration.Configuration;
import org.apache.flink.connector.prometheus.sink.PrometheusRequestSigner;
import org.apache.flink.connector.prometheus.table.PrometheusConfig;

import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;

import static org.apache.flink.connector.aws.config.AWSConfigConstants.AWS_REGION;
import static org.apache.flink.connector.prometheus.table.PrometheusConnectorOption.METRIC_REMOTE_WRITE_URL;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertInstanceOf;
import static org.junit.jupiter.api.Assertions.assertThrows;

class AmazonManagedPrometheusWriteRequestSignerFactoryTest {
private AmazonManagedPrometheusWriteRequestSignerFactory factory;

@BeforeEach
public void setup() {
factory = new AmazonManagedPrometheusWriteRequestSignerFactory();
}

@Test
void testIdentifier() {
assertEquals(factory.requestSignerIdentifer(), "amazon-managed-prometheus");
}

@Test
void testCreateRequestSigner() {
final String endpoint =
"https://aps-workspaces.us-east-1.amazonaws.com/workspaces/abc/api/v1/remote_write";
final String region = "us-east-1";
Configuration config = new Configuration();
config.set(METRIC_REMOTE_WRITE_URL, endpoint);
config.setString(AWS_REGION, region);

PrometheusRequestSigner requestSigner =
factory.getRequestSigner(new PrometheusConfig(config));

assertInstanceOf(AmazonManagedPrometheusWriteRequestSigner.class, requestSigner);
}

@Test
void testCreateRequestSignerFailsWithInvalidRegion() {
final String endpoint =
"https://aps-workspaces.us-east-1.amazonaws.com/workspaces/abc/api/v1/remote_write";
final String region = "invalid-region";
Configuration config = new Configuration();
config.set(METRIC_REMOTE_WRITE_URL, endpoint);
config.setString(AWS_REGION, region);

assertThrows(
IllegalArgumentException.class,
() -> factory.getRequestSigner(new PrometheusConfig(config)));
}

@Test
void testCreateRequestSignerFailsWithInvalidURL() {
Configuration config = new Configuration();
config.set(METRIC_REMOTE_WRITE_URL, "invalid-endpoint");
config.setString(AWS_REGION, "us-east-1");

assertThrows(
IllegalArgumentException.class,
() -> factory.getRequestSigner(new PrometheusConfig(config)));
}
}
25 changes: 25 additions & 0 deletions flink-connector-prometheus/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,31 @@ under the License.
<artifactId>httpcore5</artifactId>
</dependency>

<!--Table API dependencies-->
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-common</artifactId>
<version>${flink.version}</version>
<type>test-jar</type>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-api-java-bridge</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-table-runtime</artifactId>
<version>${flink.version}</version>
<scope>provided</scope>
<optional>true</optional>
</dependency>

<!-- test -->
<dependency>
<groupId>org.apache.flink</groupId>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,19 +37,20 @@

/** Sink implementation accepting {@link PrometheusTimeSeries} as inputs. */
@PublicEvolving
public class PrometheusSink extends AsyncSinkBase<PrometheusTimeSeries, Types.TimeSeries> {
public class PrometheusSink<IN> extends AsyncSinkBase<IN, Types.TimeSeries> {
private final String prometheusRemoteWriteUrl;
private final PrometheusAsyncHttpClientBuilder clientBuilder;
private final PrometheusRequestSigner requestSigner;
private final int maxBatchSizeInSamples;
private final int maxRecordSizeInSamples;
private final String httpUserAgent;
private final PrometheusSinkConfiguration.SinkWriterErrorHandlingBehaviorConfiguration
errorHandlingBehaviorConfig;
private final String metricGroupName;

@SuppressWarnings("checkstyle:RegexpSingleline")
protected PrometheusSink(
ElementConverter<PrometheusTimeSeries, Types.TimeSeries> elementConverter,
ElementConverter<IN, Types.TimeSeries> elementConverter,
int maxInFlightRequests,
int maxBufferedRequests,
int maxBatchSizeInSamples,
Expand Down Expand Up @@ -93,6 +94,7 @@ protected PrometheusSink(
Preconditions.checkArgument(
StringUtils.isNotBlank(metricGroupName), "Missing metric group name");
this.maxBatchSizeInSamples = maxBatchSizeInSamples;
this.maxRecordSizeInSamples = maxRecordSizeInSamples;
this.requestSigner = requestSigner;
this.prometheusRemoteWriteUrl = prometheusRemoteWriteUrl;
this.clientBuilder = clientBuilder;
Expand All @@ -102,22 +104,22 @@ protected PrometheusSink(
}

@Override
public StatefulSinkWriter<PrometheusTimeSeries, BufferedRequestState<Types.TimeSeries>>
createWriter(InitContext initContext) {
public StatefulSinkWriter<IN, BufferedRequestState<Types.TimeSeries>> createWriter(
InitContext initContext) {
SinkMetricsCallback metricsCallback =
new SinkMetricsCallback(
SinkMetrics.registerSinkMetrics(
initContext.metricGroup().addGroup(metricGroupName)));
CloseableHttpAsyncClient asyncHttpClient =
clientBuilder.buildAndStartClient(metricsCallback);

return new PrometheusSinkWriter(
return new PrometheusSinkWriter<>(
getElementConverter(),
initContext,
getMaxInFlightRequests(),
getMaxBufferedRequests(),
maxBatchSizeInSamples,
getMaxRecordSizeInBytes(),
maxRecordSizeInSamples,
getMaxTimeInBufferMS(),
prometheusRemoteWriteUrl,
asyncHttpClient,
Expand All @@ -128,23 +130,22 @@ protected PrometheusSink(
}

@Override
public StatefulSinkWriter<PrometheusTimeSeries, BufferedRequestState<Types.TimeSeries>>
restoreWriter(
InitContext initContext,
Collection<BufferedRequestState<Types.TimeSeries>> recoveredState) {
public StatefulSinkWriter<IN, BufferedRequestState<Types.TimeSeries>> restoreWriter(
InitContext initContext,
Collection<BufferedRequestState<Types.TimeSeries>> recoveredState) {
SinkMetricsCallback metricsCallback =
new SinkMetricsCallback(
SinkMetrics.registerSinkMetrics(
initContext.metricGroup().addGroup(metricGroupName)));
CloseableHttpAsyncClient asyncHttpClient =
clientBuilder.buildAndStartClient(metricsCallback);
return new PrometheusSinkWriter(
return new PrometheusSinkWriter<>(
getElementConverter(),
initContext,
getMaxInFlightRequests(),
getMaxBufferedRequests(),
maxBatchSizeInSamples,
getMaxRecordSizeInBytes(),
maxRecordSizeInSamples,
getMaxTimeInBufferMS(),
prometheusRemoteWriteUrl,
asyncHttpClient,
Expand All @@ -155,8 +156,8 @@ protected PrometheusSink(
recoveredState);
}

public static PrometheusSinkBuilder builder() {
return new PrometheusSinkBuilder();
public static <IN> PrometheusSinkBuilder<IN> builder() {
return new PrometheusSinkBuilder<>();
}

@Override
Expand Down
Loading