Skip to content

Commit

Permalink
[FLINK-19997] Implement an e2e test for sql-client with Confluent Reg…
Browse files Browse the repository at this point in the history
…istry Avro format

The e2e test uses testcontainers to run Kafka, Schema Registry and Local Flink cluster. Testcontainers use docker underneath, therefore it is required to run the tests.

The commit adds a helper FlinkContainer that wraps a local flink distribution and builds a temporary docker image on top of it. It exposes common tasks such as e.g. submitting sql job, it backs up the logs in case of failure etc.

This commit introduces a KafkaContainerClient, which uses Kafka client on top of a given KafkaContainer to perform common tasks such as creating topics or reading and writing records using given Kafka Serializers.

This closes apache#14085
  • Loading branch information
dawidwys committed Nov 25, 2020
1 parent efc12ca commit 40e6e16
Show file tree
Hide file tree
Showing 11 changed files with 892 additions and 78 deletions.
62 changes: 62 additions & 0 deletions flink-end-to-end-tests/flink-end-to-end-tests-common-kafka/pom.xml
Original file line number Diff line number Diff line change
Expand Up @@ -30,6 +30,13 @@ under the License.
<artifactId>flink-end-to-end-tests-common-kafka</artifactId>
<name>Flink : E2E Tests : Common Kafka</name>

<repositories>
<repository>
<id>confluent</id>
<url>https://packages.confluent.io/maven/</url>
</repository>
</repositories>

<dependencies>
<dependency>
<groupId>org.apache.flink</groupId>
Expand Down Expand Up @@ -61,6 +68,12 @@ under the License.
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
<version>2.4.1</version>
</dependency>

<!-- The following dependencies are for connector/format sql-jars that
we copy using the maven-dependency-plugin. When extending the test
to cover more connectors/formats, add a dependency here and an entry
Expand All @@ -80,6 +93,47 @@ under the License.
<version>${project.version}</version>
<scope>test</scope>
</dependency>
<dependency>
<!-- Used by maven-dependency-plugin -->
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-avro-confluent-registry</artifactId>
<version>${project.version}</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>org.testcontainers</groupId>
<artifactId>kafka</artifactId>
<version>1.15.0</version>
</dependency>

<dependency>
<!-- https://mvnrepository.com/artifact/io.confluent/kafka-avro-serializer -->
<groupId>io.confluent</groupId>
<artifactId>kafka-avro-serializer</artifactId>
<version>5.5.2</version>
<scope>test</scope>
</dependency>

<dependency>
<groupId>io.confluent</groupId>
<artifactId>kafka-schema-registry-client</artifactId>
<version>5.5.2</version>
<scope>test</scope>
<exclusions>
<exclusion>
<groupId>org.apache.kafka</groupId>
<artifactId>kafka-clients</artifactId>
</exclusion>
</exclusions>
</dependency>

<dependency>
<groupId>org.apache.avro</groupId>
<artifactId>avro</artifactId>
<version>1.10.0</version>
<scope>test</scope>
</dependency>
</dependencies>

<build>
Expand Down Expand Up @@ -123,6 +177,14 @@ under the License.
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
</artifactItem>
<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-avro-confluent-registry</artifactId>
<version>${project.version}</version>
<destFileName>avro-confluent.jar</destFileName>
<type>jar</type>
<outputDirectory>${project.build.directory}/dependencies</outputDirectory>
</artifactItem>
<artifactItem>
<groupId>org.apache.flink</groupId>
<artifactId>flink-sql-connector-kafka_${scala.binary.version}</artifactId>
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,131 @@
/*
* 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:https://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.tests.util.kafka;

import org.apache.flink.api.common.time.Deadline;

import org.apache.kafka.clients.CommonClientConfigs;
import org.apache.kafka.clients.admin.AdminClient;
import org.apache.kafka.clients.admin.NewTopic;
import org.apache.kafka.clients.consumer.Consumer;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.consumer.ConsumerRecord;
import org.apache.kafka.clients.consumer.ConsumerRecords;
import org.apache.kafka.clients.consumer.KafkaConsumer;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.Producer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.BytesDeserializer;
import org.apache.kafka.common.serialization.BytesSerializer;
import org.apache.kafka.common.serialization.Deserializer;
import org.apache.kafka.common.serialization.Serializer;
import org.apache.kafka.common.utils.Bytes;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.testcontainers.containers.KafkaContainer;

import java.io.IOException;
import java.time.Duration;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.regex.Pattern;

/**
* A utility class that exposes common methods over a {@link KafkaContainer}.
*/
public class KafkaContainerClient {
private static final Logger LOG = LoggerFactory.getLogger(KafkaContainerClient.class);
private final KafkaContainer container;

public KafkaContainerClient(KafkaContainer container) {
this.container = container;
}

public void createTopic(
int replicationFactor,
int numPartitions,
String topic) {
Map<String, Object> properties = new HashMap<>();
properties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
try (AdminClient admin = AdminClient.create(properties)) {
admin.createTopics(Collections.singletonList(new NewTopic(
topic,
numPartitions,
(short) replicationFactor)));
}
}

public <T> void sendMessages(String topic, Serializer<T> valueSerializer, T... messages) {
Properties props = new Properties();
props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
props.put(ProducerConfig.ACKS_CONFIG, "all");

try (
Producer<Bytes, T> producer = new KafkaProducer<>(
props,
new BytesSerializer(),
valueSerializer)) {
for (T message : messages) {
producer.send(new ProducerRecord<>(
topic,
message));
}
}
}

public <T> List<T> readMessages(
int expectedNumMessages,
String groupId,
String topic,
Deserializer<T> valueDeserializer) throws IOException {
Properties props = new Properties();
props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, container.getBootstrapServers());
props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId);
props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true");
props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest");

final List<T> messages = Collections.synchronizedList(new ArrayList<>(
expectedNumMessages));
try (
Consumer<Bytes, T> consumer = new KafkaConsumer<>(
props,
new BytesDeserializer(),
valueDeserializer)) {
consumer.subscribe(Pattern.compile(topic));
final Deadline deadline = Deadline.fromNow(Duration.ofSeconds(120));
while (deadline.hasTimeLeft() && messages.size() < expectedNumMessages) {
LOG.info("Waiting for messages. Received {}/{}.", messages.size(),
expectedNumMessages);
ConsumerRecords<Bytes, T> records = consumer.poll(Duration.ofMillis(100));
for (ConsumerRecord<Bytes, T> record : records) {
messages.add(record.value());
}
}
if (messages.size() != expectedNumMessages) {
throw new IOException("Could not read expected number of messages.");
}
return messages;
}
}
}
Loading

0 comments on commit 40e6e16

Please sign in to comment.