Skip to content
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

[NOT FOR MERGE] Kafka to pub sub e2e run actual pipeline #9

Closed
wants to merge 16 commits into from
Closed
Show file tree
Hide file tree
Changes from 15 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
Expand Up @@ -616,6 +616,7 @@ class BeamModulePlugin implements Plugin<Project> {
testcontainers_kafka : "org.testcontainers:kafka:$testcontainers_version",
testcontainers_localstack : "org.testcontainers:localstack:$testcontainers_version",
testcontainers_postgresql : "org.testcontainers:postgresql:$testcontainers_version",
testcontainers_gcloud : "org.testcontainers:gcloud:$testcontainers_version",
KhaninArtur marked this conversation as resolved.
Show resolved Hide resolved
vendored_bytebuddy_1_10_8 : "org.apache.beam:beam-vendor-bytebuddy-1_10_8:0.1",
vendored_grpc_1_26_0 : "org.apache.beam:beam-vendor-grpc-1_26_0:0.3",
vendored_guava_26_0_jre : "org.apache.beam:beam-vendor-guava-26_0-jre:0.1",
Expand Down
3 changes: 3 additions & 0 deletions examples/java/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -56,6 +56,7 @@ dependencies {
compile library.java.vendored_guava_26_0_jre
compile library.java.kafka_clients
compile project(path: ":sdks:java:core", configuration: "shadow")
compile project(path: ":runners:direct-java", configuration: "shadow")
compile project(":sdks:java:extensions:google-cloud-platform-core")
compile project(":sdks:java:io:google-cloud-platform")
compile project(":sdks:java:io:kafka")
Expand All @@ -81,6 +82,8 @@ dependencies {
testCompile library.java.hamcrest_library
testCompile library.java.junit
testCompile library.java.mockito_core
testCompile library.java.testcontainers_kafka
testCompile library.java.testcontainers_gcloud

// Add dependencies for the PreCommit configurations
// For each runner a project level dependency on the examples project.
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@
import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.configureSsl;
import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.getKafkaCredentialsFromVault;
import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.isSslSpecified;
import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.parseKafkaConsumerConfig;
import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;

import java.util.ArrayList;
Expand Down Expand Up @@ -151,17 +152,19 @@ public static void main(String[] args) {
KafkaToPubsubOptions options =
PipelineOptionsFactory.fromArgs(args).withValidation().as(KafkaToPubsubOptions.class);

run(options);
// Create the pipeline
Pipeline pipeline = Pipeline.create(options);
run(pipeline, options);
}

/**
* Runs a pipeline which reads message from Kafka and writes it to GCS.
*
* @param options arguments to the pipeline
*/
public static PipelineResult run(KafkaToPubsubOptions options) {
public static PipelineResult run(Pipeline pipeline, KafkaToPubsubOptions options) {
// Configure Kafka consumer properties
Map<String, Object> kafkaConfig = new HashMap<>();
Map<String, Object> kafkaConfig = parseKafkaConsumerConfig(options.getKafkaConsumerConfig());
KhaninArtur marked this conversation as resolved.
Show resolved Hide resolved
Map<String, String> sslConfig = new HashMap<>();
if (options.getSecretStoreUrl() != null && options.getVaultToken() != null) {
Map<String, Map<String, String>> credentials =
Expand Down Expand Up @@ -194,8 +197,6 @@ public static PipelineResult run(KafkaToPubsubOptions options) {
bootstrapServersList.size() > 0 && topicsList.get(0).length() > 0,
"bootstrapServers cannot be an empty string.");

// Create the pipeline
Pipeline pipeline = Pipeline.create(options);
LOG.info(
"Starting Kafka-To-PubSub pipeline with parameters bootstrap servers:"
+ options.getBootstrapServers()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,11 +22,14 @@
import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.USERNAME;

import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.Map;
import java.util.stream.Collectors;
import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonObject;
import org.apache.beam.vendor.grpc.v1p26p0.com.google.gson.JsonParser;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.http.HttpResponse;
import org.apache.http.client.HttpClient;
import org.apache.http.client.methods.HttpGet;
Expand Down Expand Up @@ -162,4 +165,11 @@ public static boolean isSslSpecified(KafkaToPubsubOptions options) {
|| options.getKeystorePath() != null
|| options.getKeyPassword() != null;
}

public static Map<String, Object> parseKafkaConsumerConfig(String kafkaConsumerConfig) {
return Arrays.stream(kafkaConsumerConfig.split(";"))
.map(s -> s.split("="))
.map(kv -> Pair.of(kv[0], kv[1]))
.collect(Collectors.toMap(Pair::getKey, Pair::getValue));
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -87,4 +87,9 @@ public interface KafkaToPubsubOptions extends PipelineOptions {
String getKeyPassword();

void setKeyPassword(String keyPassword);

@Description("Additional kafka consumer configs to be applied to Kafka Consumer (e.g. key1=value1;key2=value2).")
String getKafkaConsumerConfig();

void setKafkaConsumerConfig(String kafkaConfig);
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,119 @@
/*
* 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.beam.examples.complete.kafkatopubsub;

import static org.hamcrest.Matchers.equalTo;
import static org.hamcrest.Matchers.hasProperty;

import com.google.auth.Credentials;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.util.UUID;
import java.util.concurrent.ExecutionException;
import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions;
import org.apache.beam.examples.complete.kafkatopubsub.transforms.FormatTransform.FORMAT;
import org.apache.beam.runners.direct.DirectOptions;
import org.apache.beam.sdk.PipelineResult;
import org.apache.beam.sdk.extensions.gcp.auth.NoopCredentialFactory;
import org.apache.beam.sdk.extensions.gcp.options.GcpOptions;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubOptions;
import org.apache.beam.sdk.io.gcp.pubsub.TestPubsub;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap;
import org.apache.kafka.clients.consumer.ConsumerConfig;
import org.apache.kafka.clients.producer.KafkaProducer;
import org.apache.kafka.clients.producer.ProducerConfig;
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;
import org.joda.time.Duration;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.ClassRule;
import org.junit.Rule;
import org.junit.Test;
import org.testcontainers.containers.KafkaContainer;
import org.testcontainers.containers.PubSubEmulatorContainer;
import org.testcontainers.utility.DockerImageName;

/** E2E test for {@link KafkaToPubsub} pipeline. */
public class KafkaToPubsubE2ETest {
private static final String PUBSUB_EMULATOR_IMAGE =
"gcr.io/google.com/cloudsdktool/cloud-sdk:316.0.0-emulators";
private static final String KAFKA_IMAGE_NAME = "confluentinc/cp-kafka:5.4.3";
private static final String PUBSUB_MESSAGE = "test pubsub message";
private static final String KAFKA_TOPIC_NAME = "messages-topic";
private static final String PROJECT_ID = "try-kafka-pubsub";
private static final PipelineOptions OPTIONS = TestPipeline.testingPipelineOptions();

@ClassRule public static final PubSubEmulatorContainer pubSubEmulatorContainer = new PubSubEmulatorContainer(DockerImageName.parse(PUBSUB_EMULATOR_IMAGE));
@ClassRule public static final KafkaContainer kafkaContainer = new KafkaContainer(DockerImageName.parse(KAFKA_IMAGE_NAME));
@Rule public final transient TestPipeline pipeline = TestPipeline.fromOptions(OPTIONS);
@Rule public final transient TestPubsub testPubsub = TestPubsub.fromOptions(OPTIONS);

@BeforeClass
public static void beforeClass() {
Credentials credentials = NoopCredentialFactory.fromOptions(OPTIONS).getCredential();
OPTIONS.as(DirectOptions.class).setBlockOnRun(false);
OPTIONS.as(GcpOptions.class).setGcpCredential(credentials);
OPTIONS.as(GcpOptions.class).setProject(PROJECT_ID);
OPTIONS.as(PubsubOptions.class).setPubsubRootUrl("http://" + pubSubEmulatorContainer.getEmulatorEndpoint());
OPTIONS.as(KafkaToPubsubOptions.class).setOutputFormat(FORMAT.PUBSUB);
OPTIONS.as(KafkaToPubsubOptions.class).setBootstrapServers(kafkaContainer.getBootstrapServers());
OPTIONS.as(KafkaToPubsubOptions.class).setInputTopics(KAFKA_TOPIC_NAME);
OPTIONS.as(KafkaToPubsubOptions.class).setKafkaConsumerConfig(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + "=earliest");
}

@Before
public void setUp() {
OPTIONS.as(KafkaToPubsubOptions.class).setOutputTopic(testPubsub.topicPath().getPath());
}

@Test
public void testKafkaToPubsubE2E() throws IOException, InterruptedException {
PipelineResult job = KafkaToPubsub.run(pipeline, OPTIONS.as(KafkaToPubsubOptions.class));

sendKafkaMessage();
testPubsub.assertThatTopicEventuallyReceives(
hasProperty("payload", equalTo(PUBSUB_MESSAGE.getBytes(StandardCharsets.UTF_8)))
).waitForUpTo(Duration.standardSeconds(25));
KhaninArtur marked this conversation as resolved.
Show resolved Hide resolved
try {
job.cancel();
} catch (UnsupportedOperationException e) {
throw new AssertionError("Could not stop pipeline.", e);
}
}

private void sendKafkaMessage() {
try (KafkaProducer<String, String> producer =
new KafkaProducer<>(
ImmutableMap.of(
ProducerConfig.BOOTSTRAP_SERVERS_CONFIG,
kafkaContainer.getBootstrapServers(),
ProducerConfig.CLIENT_ID_CONFIG,
UUID.randomUUID().toString()),
new StringSerializer(),
new StringSerializer()
)
) {
producer.send(new ProducerRecord<>(KAFKA_TOPIC_NAME, "testcontainers", PUBSUB_MESSAGE)).get();
} catch (ExecutionException | InterruptedException e) {
throw new RuntimeException("Something went wrong in kafka producer", e);
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,9 @@

import com.google.api.core.ApiFuture;
import com.google.api.core.ApiFutures;
import com.google.api.gax.grpc.GrpcTransportChannel;
import com.google.api.gax.rpc.FixedTransportChannelProvider;
import com.google.api.gax.rpc.TransportChannelProvider;
import com.google.cloud.pubsub.v1.AckReplyConsumer;
import com.google.cloud.pubsub.v1.MessageReceiver;
import com.google.cloud.pubsub.v1.Publisher;
Expand All @@ -32,6 +35,8 @@
import com.google.cloud.pubsub.v1.TopicAdminSettings;
import com.google.protobuf.ByteString;
import com.google.pubsub.v1.PushConfig;
import io.grpc.ManagedChannel;
import io.grpc.ManagedChannelBuilder;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.BlockingQueue;
Expand All @@ -42,6 +47,7 @@
import java.util.stream.Collectors;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.SubscriptionPath;
import org.apache.beam.sdk.io.gcp.pubsub.PubsubClient.TopicPath;
import org.apache.beam.sdk.options.PipelineOptions;
import org.apache.beam.sdk.testing.TestPipeline;
import org.apache.beam.sdk.testing.TestPipelineOptions;
import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions;
Expand Down Expand Up @@ -77,25 +83,37 @@ public class TestPubsub implements TestRule {

private final TestPubsubOptions pipelineOptions;
private final String pubsubEndpoint;
private final boolean isLocalhost;

private @Nullable TopicAdminClient topicAdmin = null;
private @Nullable SubscriptionAdminClient subscriptionAdmin = null;
private @Nullable TopicPath eventsTopicPath = null;
private @Nullable SubscriptionPath subscriptionPath = null;
private @Nullable ManagedChannel channel = null;
private @Nullable TransportChannelProvider channelProvider = null;

/**
* Creates an instance of this rule.
* Creates an instance of this rule using options provided by {@link TestPipeline#testingPipelineOptions()}.
*
* <p>Loads GCP configuration from {@link TestPipelineOptions}.
*/
public static TestPubsub create() {
TestPubsubOptions options = TestPipeline.testingPipelineOptions().as(TestPubsubOptions.class);
return new TestPubsub(options);
return fromOptions(TestPipeline.testingPipelineOptions());
}

/**
* Creates an instance of this rule using provided options.
*
* <p>Loads GCP configuration from {@link TestPipelineOptions}.
*/
public static TestPubsub fromOptions(PipelineOptions options) {
return new TestPubsub(options.as(TestPubsubOptions.class));
}

private TestPubsub(TestPubsubOptions pipelineOptions) {
this.pipelineOptions = pipelineOptions;
this.pubsubEndpoint = PubsubOptions.targetForRootUrl(this.pipelineOptions.getPubsubRootUrl());
this.isLocalhost = this.pubsubEndpoint.startsWith("localhost");
}

@Override
Expand Down Expand Up @@ -124,16 +142,24 @@ public void evaluate() throws Throwable {
}

private void initializePubsub(Description description) throws IOException {
if (isLocalhost) {
channel = ManagedChannelBuilder.forTarget(pubsubEndpoint).usePlaintext().build();
} else {
channel = ManagedChannelBuilder.forTarget(pubsubEndpoint).useTransportSecurity().build();
}
channelProvider = FixedTransportChannelProvider.create(GrpcTransportChannel.create(channel));
topicAdmin =
TopicAdminClient.create(
TopicAdminSettings.newBuilder()
.setCredentialsProvider(pipelineOptions::getGcpCredential)
.setTransportChannelProvider(channelProvider)
.setEndpoint(pubsubEndpoint)
.build());
subscriptionAdmin =
SubscriptionAdminClient.create(
SubscriptionAdminSettings.newBuilder()
.setCredentialsProvider(pipelineOptions::getGcpCredential)
.setTransportChannelProvider(channelProvider)
.setEndpoint(pubsubEndpoint)
.build());
TopicPath eventsTopicPathTmp =
Expand All @@ -160,8 +186,8 @@ private void initializePubsub(Description description) throws IOException {
subscriptionPath = subscriptionPathTmp;
}

private void tearDown() throws IOException {
if (subscriptionAdmin == null || topicAdmin == null) {
private void tearDown() {
if (subscriptionAdmin == null || topicAdmin == null || channel == null) {
return;
}

Expand All @@ -179,9 +205,12 @@ private void tearDown() throws IOException {
} finally {
subscriptionAdmin.close();
topicAdmin.close();
channel.shutdown();

subscriptionAdmin = null;
topicAdmin = null;
channelProvider = null;
channel = null;

eventsTopicPath = null;
subscriptionPath = null;
Expand Down Expand Up @@ -245,6 +274,7 @@ public void publish(List<PubsubMessage> messages) {
eventPublisher =
Publisher.newBuilder(eventsTopicPath.getPath())
.setCredentialsProvider(pipelineOptions::getGcpCredential)
.setChannelProvider(channelProvider)
.setEndpoint(pubsubEndpoint)
.build();
} catch (IOException e) {
Expand Down Expand Up @@ -297,6 +327,7 @@ public List<PubsubMessage> waitForNMessages(int n, Duration timeoutDuration)
Subscriber subscriber =
Subscriber.newBuilder(subscriptionPath.getPath(), receiver)
.setCredentialsProvider(pipelineOptions::getGcpCredential)
.setChannelProvider(channelProvider)
.setEndpoint(pubsubEndpoint)
.build();
subscriber.startAsync();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,7 @@
* <p>Uses a random temporary Pubsub topic for synchronization.
*/
@SuppressWarnings({
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
"nullness" // TODO(https://issues.apache.org/jira/browse/BEAM-10402)
KhaninArtur marked this conversation as resolved.
Show resolved Hide resolved
})
public class TestPubsubSignal implements TestRule {
private static final Logger LOG = LoggerFactory.getLogger(TestPubsubSignal.class);
Expand Down