Skip to content

[FLINK-37688] Implement Amazon CloudWatch Metric Sink Connector #202

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
Original file line number Diff line number Diff line change
@@ -0,0 +1,141 @@
<?xml version="1.0" encoding="UTF-8"?>
<!--
~ 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.
-->

<project xmlns="http://maven.apache.org/POM/4.0.0"
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">

<parent>
<artifactId>flink-connector-aws-e2e-tests-parent</artifactId>
<groupId>org.apache.flink</groupId>
<version>5.1-SNAPSHOT</version>
</parent>

<modelVersion>4.0.0</modelVersion>

<artifactId>flink-connector-aws-cloudwatch-e2e-tests</artifactId>
<name>Flink : Connectors : AWS : E2E Tests : Amazon CloudWatch</name>
<packaging>jar</packaging>

<properties>
<aws.sdkv2.version>2.31.18</aws.sdkv2.version>
</properties>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-streaming-java</artifactId>
<version>${flink.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-cloudwatch</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-cloudwatch</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-aws-base</artifactId>
<version>${project.version}</version>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-aws-base</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
</dependency>

<dependency>
<groupId>org.apache.flink</groupId>
<artifactId>flink-connector-cloudwatch</artifactId>
<version>${project.version}</version>
<type>test-jar</type>
</dependency>

<!-- Other third-party dependencies -->
<dependency>
<groupId>com.google.guava</groupId>
<artifactId>guava</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.core</groupId>
<artifactId>jackson-databind</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>com.fasterxml.jackson.datatype</groupId>
<artifactId>jackson-datatype-jsr310</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>s3</artifactId>
<scope>test</scope>
</dependency>

<dependency>
<groupId>software.amazon.awssdk</groupId>
<artifactId>cloudwatch</artifactId>
<scope>test</scope>
</dependency>
</dependencies>

<build>
<plugins>
<plugin>
<groupId>org.apache.maven.plugins</groupId>
<artifactId>maven-dependency-plugin</artifactId>
<executions>
<execution>
<id>copy</id>
<phase>pre-integration-test</phase>
<goals>
<goal>copy</goal>
</goals>
</execution>
</executions>
<configuration>
<artifactItems>
<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-cloudwatch</artifactId>
<version>${project.version}</version>
<destFileName>sql-cloudwatch.jar</destFileName>
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
</artifactItem>
</artifactItems>
</configuration>
</plugin>
</plugins>
</build>
</project>
Original file line number Diff line number Diff line change
@@ -0,0 +1,157 @@
/*
* 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.cloudwatch.sink.test;

import org.apache.flink.connector.aws.testutils.AWSServicesTestUtils;
import org.apache.flink.connector.aws.testutils.LocalstackContainer;
import org.apache.flink.connector.aws.util.AWSGeneralUtil;
import org.apache.flink.connector.cloudwatch.sink.CloudWatchSink;
import org.apache.flink.connector.cloudwatch.sink.MetricWriteRequest;
import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment;
import org.apache.flink.test.junit5.MiniClusterExtension;

import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.api.extension.ExtendWith;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.Network;
import org.testcontainers.junit.jupiter.Container;
import org.testcontainers.junit.jupiter.Testcontainers;
import org.testcontainers.utility.DockerImageName;
import software.amazon.awssdk.core.SdkSystemSetting;
import software.amazon.awssdk.http.SdkHttpClient;
import software.amazon.awssdk.services.cloudwatch.CloudWatchClient;
import software.amazon.awssdk.services.cloudwatch.model.GetMetricDataRequest;
import software.amazon.awssdk.services.cloudwatch.model.GetMetricDataResponse;
import software.amazon.awssdk.services.cloudwatch.model.Metric;
import software.amazon.awssdk.services.cloudwatch.model.MetricDataQuery;
import software.amazon.awssdk.services.cloudwatch.model.MetricStat;

import java.time.Instant;
import java.util.UUID;

import static org.apache.flink.connector.aws.testutils.AWSServicesTestUtils.createConfig;
import static org.assertj.core.api.Assertions.assertThat;

/** Integration test for {@link CloudWatchSink}. */
@Testcontainers
@ExtendWith(MiniClusterExtension.class)
public class CloudWatchSinkITCase {
private static final Logger LOG = LoggerFactory.getLogger(CloudWatchSinkITCase.class);

private static String testMetricName;
private static final int NUMBER_OF_ELEMENTS = 50;

private static StreamExecutionEnvironment env;

private CloudWatchClient cloudWatchClient;
private SdkHttpClient httpClient;
private static final Network network = Network.newNetwork();
private static final String LOCALSTACK_DOCKER_IMAGE_VERSION = "localstack/localstack:3.7.2";
private static final String TEST_NAMESPACE = "test_namespace";

@Container
private static final LocalstackContainer MOCK_CLOUDWATCH_CONTAINER =
new LocalstackContainer(DockerImageName.parse(LOCALSTACK_DOCKER_IMAGE_VERSION))
.withNetwork(network)
.withNetworkAliases("localstack");

@BeforeEach
public void setup() {
System.setProperty(SdkSystemSetting.CBOR_ENABLED.property(), "false");

testMetricName = UUID.randomUUID().toString();

env = StreamExecutionEnvironment.getExecutionEnvironment();
env.setParallelism(1);

httpClient = AWSServicesTestUtils.createHttpClient();

cloudWatchClient =
AWSServicesTestUtils.createAwsSyncClient(
MOCK_CLOUDWATCH_CONTAINER.getEndpoint(),
httpClient,
CloudWatchClient.builder());

LOG.info("Done setting up the localstack.");
}

@AfterEach
public void teardown() {
System.clearProperty(SdkSystemSetting.CBOR_ENABLED.property());
AWSGeneralUtil.closeResources(httpClient, cloudWatchClient);
}

@Test
public void testRandomDataSuccessfullyWritten() throws Exception {
CloudWatchSink<MetricWriteRequest> cloudWatchSink =
CloudWatchSink.<MetricWriteRequest>builder()
.setNamespace(TEST_NAMESPACE)
.setCloudWatchClientProperties(
createConfig(MOCK_CLOUDWATCH_CONTAINER.getEndpoint()))
.build();

Instant testTimestamp = Instant.now();

env.fromSequence(1, NUMBER_OF_ELEMENTS)
.map(
data ->
MetricWriteRequest.builder()
.withMetricName(testMetricName)
.addValue(1.0d)
.withTimestamp(testTimestamp)
.build())
.sinkTo(cloudWatchSink);

env.execute("Integration Test");

GetMetricDataResponse response =
cloudWatchClient.getMetricData(
GetMetricDataRequest.builder()
.metricDataQueries(
MetricDataQuery.builder()
.metricStat(getMetricStat("Sum"))
.build(),
MetricDataQuery.builder()
.metricStat(getMetricStat("SampleCount"))
.build())
.startTime(testTimestamp.minusSeconds(300))
.endTime(testTimestamp.plusSeconds(300))
.build());

response.metricDataResults()
.forEach(
result ->
assertThat(result.values())
.containsExactly(Double.valueOf(NUMBER_OF_ELEMENTS)));
}

private static MetricStat getMetricStat(String stat) {
return MetricStat.builder()
.metric(
Metric.builder()
.namespace(TEST_NAMESPACE)
.metricName(testMetricName)
.build())
.stat(stat)
.period(300)
.build();
}
}
Loading