From 45ef0091df248595f6da944c399857ed71c27a56 Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 25 Sep 2020 15:52:49 +0300 Subject: [PATCH 01/73] add initial template and dependencies --- examples/java/build.gradle | 4 +- .../examples/templates/KafkaToPubsub.java | 150 ++++++++++++++++++ 2 files changed, 153 insertions(+), 1 deletion(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java diff --git a/examples/java/build.gradle b/examples/java/build.gradle index eaad35f85937..5f3eac99c582 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -48,11 +48,13 @@ configurations.sparkRunnerPreCommit { } dependencies { - compile library.java.vendored_guava_26_0_jre + provided library.java.kafka_clients compile project(path: ":sdks:java:core", 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") compile project(":sdks:java:extensions:ml") + compile project(":sdks:java:options:PipelineOptions") compile library.java.avro compile library.java.bigdataoss_util compile library.java.google_api_client diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java new file mode 100644 index 000000000000..0bd872d8e258 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java @@ -0,0 +1,150 @@ +package org.apache.beam.examples.templates; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.options.*; +import org.apache.beam.sdk.transforms.Values; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class KafkaToPubsub { + /* + * The {@link KafkaToPubSub} pipeline is a streaming pipeline which ingests data in JSON format + * from Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap servers + * are specified by the user as template parameters. + * + *

Pipeline Requirements + * + *

+ * + *

Example Usage + * + *

+     * # Set the pipeline vars
+     * PROJECT_ID=PROJECT ID HERE
+     * BUCKET_NAME=BUCKET NAME HERE
+     * PIPELINE_FOLDER=gs://${BUCKET_NAME}/dataflow/pipelines/kafka-to-pubsub
+     *
+     * # Set the runner
+     * RUNNER=DataflowRunner
+     *
+     * # Build the template
+     * mvn compile exec:java \
+     * -Dexec.mainClass=com.google.cloud.teleport.templates.KafkaToPubsub \
+     * -Dexec.cleanupDaemonThreads=false \
+     * -Dexec.args=" \
+     * --project=${PROJECT_ID} \
+     * --stagingLocation=${PIPELINE_FOLDER}/staging \
+     * --tempLocation=${PIPELINE_FOLDER}/temp \
+     * --templateLocation=${PIPELINE_FOLDER}/template \
+     * --runner=${RUNNER}"
+     *
+     * # Execute the template
+     * JOB_NAME=kafka-to-pubsub-$USER-`date +"%Y%m%d-%H%M%S%z"`
+     *
+     * gcloud dataflow jobs run ${JOB_NAME} \
+     * --gcs-location=${PIPELINE_FOLDER}/template \
+     * --zone=us-east1-d \
+     * --parameters \
+     * "bootstrapServers=my_host:9092,inputTopic=kafka-test,\
+     * outputPubsubTopic=pubsub-test"
+     * 
+ */ + + /** + * The log to output status messages to. + */ + private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubsub.class); + + + /** + * The {@link Options} class provides the custom execution options passed by the executor at the + * command-line. + */ + public interface Options extends PipelineOptions { + + @Description("Kafka Bootstrap Servers") + ValueProvider getBootstrapServers(); + + void setBootstrapServers(ValueProvider value); + + @Description("Kafka topic to read the input from") + ValueProvider getInputTopic(); + + void setInputTopic(ValueProvider value); + + @Description( + "The Cloud Pub/Sub topic to publish to. " + + "The name should be in the format of " + + "projects//topics/.") + @Validation.Required + ValueProvider getOutputTopic(); + + void setOutputTopic(ValueProvider outputTopic); + } + + /** + * The main entry-point for pipeline execution. This method will start the pipeline but will not + * wait for it's execution to finish. If blocking execution is required, use the {@link + * KafkaToPubsub#run(Options)} method to start the pipeline and invoke {@code + * result.waitUntilFinish()} on the {@link PipelineResult}. + * + * @param args The command-line args passed by the executor. + */ + public static void main(String[] args) { + Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); + + run(options); + } + + /** + * Runs the pipeline to completion with the specified options. This method does not wait until the + * pipeline is finished before returning. Invoke {@code result.waitUntilFinish()} on the result + * object to block until the pipeline is finished running if blocking programmatic execution is + * required. + * + * @param options The execution options. + * @return The pipeline result. + */ + public static PipelineResult run(Options options) { + + Pipeline pipeline = Pipeline.create(options); + + // Register the coder for pipeline + + + /* + * Steps: + * 1) Read messages in from Kafka + * 3) Write successful records to PubSub + */ + pipeline.apply( + "ReadFromKafka", + KafkaIO.read() + .withBootstrapServers(options.getBootstrapServers().get()) + .withTopic(options.getInputTopic().get()) + .withKeyDeserializer(StringDeserializer.class) + .withValueDeserializer(StringDeserializer.class) + // NumSplits is hard-coded to 1 for single-partition use cases (e.g., Debezium + // Change Data Capture). Once Dataflow dynamic templates are available, this can + // be deprecated. + .withoutMetadata()) + .apply(Values.create()) + .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); + + + return pipeline.run(); + } + + +} + + + + From 321c7158f61765c8d515f20e607dbb7ef07473a4 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Sun, 27 Sep 2020 18:23:49 +0300 Subject: [PATCH 02/73] Added flex template creation with metadata support and instructions --- examples/java/build.gradle | 10 +- .../examples/templates/KafkaToPubSub.java | 157 ++++++++++++++++++ .../examples/templates/KafkaToPubsub.java | 150 ----------------- .../templates/kafka_to_pubsub_metadata.json | 27 +++ 4 files changed, 190 insertions(+), 154 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 5f3eac99c582..6579d47ba3f3 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -18,7 +18,10 @@ import groovy.json.JsonOutput -plugins { id 'org.apache.beam.module' } +plugins { + id 'org.apache.beam.module' + id 'com.github.johnrengelman.shadow' +} applyJavaNature( exportJavadoc: false, enableChecker: false, @@ -48,13 +51,13 @@ configurations.sparkRunnerPreCommit { } dependencies { - provided library.java.kafka_clients + compile library.java.kafka_clients compile project(path: ":sdks:java:core", 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") compile project(":sdks:java:extensions:ml") - compile project(":sdks:java:options:PipelineOptions") + compile project(":runners:google-cloud-dataflow-java") compile library.java.avro compile library.java.bigdataoss_util compile library.java.google_api_client @@ -130,4 +133,3 @@ task preCommit() { dependsOn runner + "PreCommit" } } - diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java new file mode 100644 index 000000000000..f07d36049a3b --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java @@ -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.beam.examples.templates; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.options.Validation; +import org.apache.beam.sdk.transforms.Values; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link KafkaToPubSub} pipeline is a streaming pipeline which ingests data in JSON format from + * Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap servers + * are specified by the user as template parameters. + * + *

Pipeline Requirements + * + *

    + *
  • The Kafka topic exists and the message is encoded in a valid JSON format. + *
  • The PubSub output topic exists. + *
+ * + *

Example Usage + * + *

+ * # Set the pipeline vars
+ * export PROJECT=ID_OF_MY_PROJECT
+ * export BUCKET_NAME=MY_BUCKET
+ * export TEMPLATE_PATH="gs://$BUCKET/samples/dataflow/templates/kafka-pubsub.json"
+ * export TEMPLATE_IMAGE="gcr.io/$PROJECT/samples/dataflow/kafka-pubsub:latest"
+ *
+ * # Go to the beam folder
+ * cd /path/to/beam
+ *
+ * # Create bucket in the cloud storage
+ * gsutil mb gs://${BUCKET_NAME}
+ *
+ * FLEX TEMPLATE
+ * # Assemble uber-jar
+ * ./gradlew -p examples/java clean shadowJar
+ *
+ * # Build the flex template
+ * gcloud dataflow flex-template build $TEMPLATE_PATH \
+ *       --image-gcr-path "$TEMPLATE_IMAGE" \
+ *       --sdk-language "JAVA" \
+ *       --flex-template-base-image JAVA11 \
+ *       --metadata-file "src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json" \
+ *       --jar "build/libs/beam-examples-java-2.25.0-SNAPSHOT-all.jar" \
+ *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.examples.templates.KafkaToPubSub"
+ */
+public class KafkaToPubSub {
+
+  /** The log to output status messages to. */
+  private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubSub.class);
+
+  /**
+   * The {@link Options} class provides the custom execution options passed by the executor at the
+   * command-line.
+   */
+  public interface Options extends PipelineOptions {
+
+    @Description("Kafka Bootstrap Servers")
+    @Validation.Required
+    String getBootstrapServers();
+
+    void setBootstrapServers(String value);
+
+    @Description("Kafka topic to read the input from")
+    @Validation.Required
+    String getInputTopic();
+
+    void setInputTopic(String value);
+
+    @Description(
+        "The Cloud Pub/Sub topic to publish to. "
+            + "The name should be in the format of "
+            + "projects//topics/.")
+    @Validation.Required
+    String getOutputTopic();
+
+    void setOutputTopic(String outputTopic);
+  }
+
+  /**
+   * The main entry-point for pipeline execution. This method will start the pipeline but will not
+   * wait for it's execution to finish. If blocking execution is required, use the {@link
+   * KafkaToPubSub#run(Options)} method to start the pipeline and invoke {@code
+   * result.waitUntilFinish()} on the {@link PipelineResult}.
+   *
+   * @param args The command-line args passed by the executor.
+   */
+  public static void main(String[] args) {
+    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
+
+    run(options);
+  }
+
+  /**
+   * Runs the pipeline to completion with the specified options. This method does not wait until the
+   * pipeline is finished before returning. Invoke {@code result.waitUntilFinish()} on the result
+   * object to block until the pipeline is finished running if blocking programmatic execution is
+   * required.
+   *
+   * @param options The execution options.
+   * @return The pipeline result.
+   */
+  public static PipelineResult run(Options options) {
+
+    Pipeline pipeline = Pipeline.create(options);
+
+    // Register the coder for pipeline
+
+    /*
+     * Steps:
+     *  1) Read messages in from Kafka
+     *  3) Write successful records to PubSub
+     */
+    pipeline
+        .apply(
+            "ReadFromKafka",
+            KafkaIO.read()
+                .withBootstrapServers(options.getBootstrapServers())
+                .withTopic(options.getInputTopic())
+                .withKeyDeserializer(StringDeserializer.class)
+                .withValueDeserializer(StringDeserializer.class)
+                // NumSplits is hard-coded to 1 for single-partition use cases (e.g., Debezium
+                // Change Data Capture). Once Dataflow dynamic templates are available, this can
+                // be deprecated.
+                .withoutMetadata())
+        .apply(Values.create())
+        .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic()));
+
+    return pipeline.run();
+  }
+}
diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java
deleted file mode 100644
index 0bd872d8e258..000000000000
--- a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubsub.java
+++ /dev/null
@@ -1,150 +0,0 @@
-package org.apache.beam.examples.templates;
-
-import org.apache.beam.sdk.Pipeline;
-import org.apache.beam.sdk.PipelineResult;
-import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
-import org.apache.beam.sdk.io.kafka.KafkaIO;
-import org.apache.beam.sdk.options.*;
-import org.apache.beam.sdk.transforms.Values;
-import org.apache.kafka.common.serialization.StringDeserializer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class KafkaToPubsub {
-    /*
-     * The {@link KafkaToPubSub} pipeline is a streaming pipeline which ingests data in JSON format
-     * from Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap servers
-     * are specified by the user as template parameters.
-     *
-     * 

Pipeline Requirements - * - *

    - *
  • The Kafka topic exists and the message is encoded in a valid JSON format. - *
  • The PubSub output topic exists. - *
- * - *

Example Usage - * - *

-     * # Set the pipeline vars
-     * PROJECT_ID=PROJECT ID HERE
-     * BUCKET_NAME=BUCKET NAME HERE
-     * PIPELINE_FOLDER=gs://${BUCKET_NAME}/dataflow/pipelines/kafka-to-pubsub
-     *
-     * # Set the runner
-     * RUNNER=DataflowRunner
-     *
-     * # Build the template
-     * mvn compile exec:java \
-     * -Dexec.mainClass=com.google.cloud.teleport.templates.KafkaToPubsub \
-     * -Dexec.cleanupDaemonThreads=false \
-     * -Dexec.args=" \
-     * --project=${PROJECT_ID} \
-     * --stagingLocation=${PIPELINE_FOLDER}/staging \
-     * --tempLocation=${PIPELINE_FOLDER}/temp \
-     * --templateLocation=${PIPELINE_FOLDER}/template \
-     * --runner=${RUNNER}"
-     *
-     * # Execute the template
-     * JOB_NAME=kafka-to-pubsub-$USER-`date +"%Y%m%d-%H%M%S%z"`
-     *
-     * gcloud dataflow jobs run ${JOB_NAME} \
-     * --gcs-location=${PIPELINE_FOLDER}/template \
-     * --zone=us-east1-d \
-     * --parameters \
-     * "bootstrapServers=my_host:9092,inputTopic=kafka-test,\
-     * outputPubsubTopic=pubsub-test"
-     * 
- */ - - /** - * The log to output status messages to. - */ - private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubsub.class); - - - /** - * The {@link Options} class provides the custom execution options passed by the executor at the - * command-line. - */ - public interface Options extends PipelineOptions { - - @Description("Kafka Bootstrap Servers") - ValueProvider getBootstrapServers(); - - void setBootstrapServers(ValueProvider value); - - @Description("Kafka topic to read the input from") - ValueProvider getInputTopic(); - - void setInputTopic(ValueProvider value); - - @Description( - "The Cloud Pub/Sub topic to publish to. " - + "The name should be in the format of " - + "projects//topics/.") - @Validation.Required - ValueProvider getOutputTopic(); - - void setOutputTopic(ValueProvider outputTopic); - } - - /** - * The main entry-point for pipeline execution. This method will start the pipeline but will not - * wait for it's execution to finish. If blocking execution is required, use the {@link - * KafkaToPubsub#run(Options)} method to start the pipeline and invoke {@code - * result.waitUntilFinish()} on the {@link PipelineResult}. - * - * @param args The command-line args passed by the executor. - */ - public static void main(String[] args) { - Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class); - - run(options); - } - - /** - * Runs the pipeline to completion with the specified options. This method does not wait until the - * pipeline is finished before returning. Invoke {@code result.waitUntilFinish()} on the result - * object to block until the pipeline is finished running if blocking programmatic execution is - * required. - * - * @param options The execution options. - * @return The pipeline result. - */ - public static PipelineResult run(Options options) { - - Pipeline pipeline = Pipeline.create(options); - - // Register the coder for pipeline - - - /* - * Steps: - * 1) Read messages in from Kafka - * 3) Write successful records to PubSub - */ - pipeline.apply( - "ReadFromKafka", - KafkaIO.read() - .withBootstrapServers(options.getBootstrapServers().get()) - .withTopic(options.getInputTopic().get()) - .withKeyDeserializer(StringDeserializer.class) - .withValueDeserializer(StringDeserializer.class) - // NumSplits is hard-coded to 1 for single-partition use cases (e.g., Debezium - // Change Data Capture). Once Dataflow dynamic templates are available, this can - // be deprecated. - .withoutMetadata()) - .apply(Values.create()) - .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); - - - return pipeline.run(); - } - - -} - - - - diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json b/examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json new file mode 100644 index 000000000000..84352fad6cf8 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json @@ -0,0 +1,27 @@ +{ + "name": "Kafka to Pub/Sub", + "description": "Streaming pipeline. Ingests JSON-encoded messages from Kafka to a Pub/Sub subscription", + "parameters": [ + { + "name": "bootstrapServers", + "label": "Kafka Bootstrap Servers", + "helpText": "Comma separated kafka bootstrap servers in format ip:port", + "regexes": [], + "paramType": "TEXT" + }, + { + "name": "inputTopic", + "label": "Kafka input topic", + "helpText": "Kafka topic to read data", + "regexes": [], + "paramType": "TEXT" + }, + { + "name": "outputTopic", + "label": "Pub/Sub output topic", + "helpText": "Pub/Sub topic to write the output, in the format of 'projects/yourproject/topics/yourtopic'", + "regexes": [], + "paramType": "TEXT" + } + ] +} \ No newline at end of file From e88fd31914a79223164181f151ea7ec14965ec8f Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 17:32:53 +0300 Subject: [PATCH 03/73] added new gradle modules for templates --- settings.gradle | 3 +++ templates/build.gradle | 13 ++++++++++ templates/kafka-to-pubsub/build.gradle | 34 ++++++++++++++++++++++++++ 3 files changed, 50 insertions(+) create mode 100644 templates/build.gradle create mode 100644 templates/kafka-to-pubsub/build.gradle diff --git a/settings.gradle b/settings.gradle index 36b68d5509b5..22adb380a37d 100644 --- a/settings.gradle +++ b/settings.gradle @@ -214,3 +214,6 @@ include "beam-test-tools" project(":beam-test-tools").dir = file(".test-infra/tools") include "beam-test-jenkins" project(":beam-test-jenkins").dir = file(".test-infra/jenkins") +include ':templates' +include ':templates:kafka-to-pubsub' + diff --git a/templates/build.gradle b/templates/build.gradle new file mode 100644 index 000000000000..4c49b4adf050 --- /dev/null +++ b/templates/build.gradle @@ -0,0 +1,13 @@ +plugins { + id 'java' +} + +version '2.25.0-SNAPSHOT' + +repositories { + mavenCentral() +} + +dependencies { + testCompile group: 'junit', name: 'junit', version: '4.12' +} diff --git a/templates/kafka-to-pubsub/build.gradle b/templates/kafka-to-pubsub/build.gradle new file mode 100644 index 000000000000..d5e6640e7971 --- /dev/null +++ b/templates/kafka-to-pubsub/build.gradle @@ -0,0 +1,34 @@ +plugins { + id 'java' + id 'org.apache.beam.module' + id 'com.github.johnrengelman.shadow' +} +applyJavaNature( + exportJavadoc: false, + enableChecker: false, + ignoreRawtypeErrors: true, + automaticModuleName: 'org.apache.beam.templates.kafka-to-pubsub') +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() + +description = "Apache Beam :: Templates :: Kafka to PubSub" +ext.summary = """Apache Beam SDK provides a simple, Java-based +interface for processing virtually any size data. This +artifact includes Kafka to PubSub template for Google DataFlow""" + +repositories { + mavenCentral() +} + +dependencies { + testCompile group: 'junit', name: 'junit', version: '4.12' + compile library.java.kafka_clients + compile project(path: ":sdks:java:core", 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") + compile project(":runners:google-cloud-dataflow-java") +} + +def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' +def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' From ed9be584996f3d210e55c72d7c2b7ea737c01a2e Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 17:39:25 +0300 Subject: [PATCH 04/73] moved metadata to template location, reverted examples build.gradle --- examples/java/build.gradle | 30 ++++++++----------- .../resources}/kafka_to_pubsub_metadata.json | 0 2 files changed, 13 insertions(+), 17 deletions(-) rename {examples/java/src/main/java/org/apache/beam/examples/templates => templates/kafka-to-pubsub/src/main/resources}/kafka_to_pubsub_metadata.json (100%) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 6579d47ba3f3..10bfd4a129a0 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -18,15 +18,12 @@ import groovy.json.JsonOutput -plugins { - id 'org.apache.beam.module' - id 'com.github.johnrengelman.shadow' -} +plugins { id 'org.apache.beam.module' } applyJavaNature( - exportJavadoc: false, - enableChecker: false, - ignoreRawtypeErrors: true, - automaticModuleName: 'org.apache.beam.examples') + exportJavadoc: false, + enableChecker: false, + ignoreRawtypeErrors: true, + automaticModuleName: 'org.apache.beam.examples') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() @@ -51,13 +48,11 @@ configurations.sparkRunnerPreCommit { } dependencies { - compile library.java.kafka_clients + compile library.java.vendored_guava_26_0_jre compile project(path: ":sdks:java:core", 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") compile project(":sdks:java:extensions:ml") - compile project(":runners:google-cloud-dataflow-java") compile library.java.avro compile library.java.bigdataoss_util compile library.java.google_api_client @@ -102,9 +97,9 @@ dependencies { * of integration tests for WordCount and WindowedWordCount. */ def preCommitRunnerClass = [ - directRunner: "org.apache.beam.runners.direct.DirectRunner", - flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", - sparkRunner: "org.apache.beam.runners.spark.TestSparkRunner", + directRunner: "org.apache.beam.runners.direct.DirectRunner", + flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", + sparkRunner: "org.apache.beam.runners.spark.TestSparkRunner", ] def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' @@ -112,9 +107,9 @@ def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for- for (String runner : preCommitRunners) { tasks.create(name: runner + "PreCommit", type: Test) { def preCommitBeamTestPipelineOptions = [ - "--project=${gcpProject}", - "--tempRoot=${gcsTempRoot}", - "--runner=" + preCommitRunnerClass[runner], + "--project=${gcpProject}", + "--tempRoot=${gcsTempRoot}", + "--runner=" + preCommitRunnerClass[runner], ] classpath = configurations."${runner}PreCommit" include "**/WordCountIT.class" @@ -133,3 +128,4 @@ task preCommit() { dependsOn runner + "PreCommit" } } + diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json rename to templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json From e2a8317b55d3969ead0b58fd8d11f68bec1db40e Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 17:40:15 +0300 Subject: [PATCH 05/73] Moved KafkaToPubsub to template, implemented options in separate package --- .../examples/templates/KafkaToPubSub.java | 157 --------------- .../apache/beam/templates/KafkaToPubsub.java | 185 ++++++++++++++++++ .../options/KafkaToPubsubOptions.java | 45 +++++ 3 files changed, 230 insertions(+), 157 deletions(-) delete mode 100644 examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java create mode 100644 templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java create mode 100644 templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java b/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java deleted file mode 100644 index f07d36049a3b..000000000000 --- a/examples/java/src/main/java/org/apache/beam/examples/templates/KafkaToPubSub.java +++ /dev/null @@ -1,157 +0,0 @@ -/* - * 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.templates; - -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; -import org.apache.beam.sdk.io.kafka.KafkaIO; -import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; -import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.options.Validation; -import org.apache.beam.sdk.transforms.Values; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * The {@link KafkaToPubSub} pipeline is a streaming pipeline which ingests data in JSON format from - * Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap servers - * are specified by the user as template parameters. - * - *

Pipeline Requirements - * - *

    - *
  • The Kafka topic exists and the message is encoded in a valid JSON format. - *
  • The PubSub output topic exists. - *
- * - *

Example Usage - * - *

- * # Set the pipeline vars
- * export PROJECT=ID_OF_MY_PROJECT
- * export BUCKET_NAME=MY_BUCKET
- * export TEMPLATE_PATH="gs://$BUCKET/samples/dataflow/templates/kafka-pubsub.json"
- * export TEMPLATE_IMAGE="gcr.io/$PROJECT/samples/dataflow/kafka-pubsub:latest"
- *
- * # Go to the beam folder
- * cd /path/to/beam
- *
- * # Create bucket in the cloud storage
- * gsutil mb gs://${BUCKET_NAME}
- *
- * FLEX TEMPLATE
- * # Assemble uber-jar
- * ./gradlew -p examples/java clean shadowJar
- *
- * # Build the flex template
- * gcloud dataflow flex-template build $TEMPLATE_PATH \
- *       --image-gcr-path "$TEMPLATE_IMAGE" \
- *       --sdk-language "JAVA" \
- *       --flex-template-base-image JAVA11 \
- *       --metadata-file "src/main/java/org/apache/beam/examples/templates/kafka_to_pubsub_metadata.json" \
- *       --jar "build/libs/beam-examples-java-2.25.0-SNAPSHOT-all.jar" \
- *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.examples.templates.KafkaToPubSub"
- */
-public class KafkaToPubSub {
-
-  /** The log to output status messages to. */
-  private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubSub.class);
-
-  /**
-   * The {@link Options} class provides the custom execution options passed by the executor at the
-   * command-line.
-   */
-  public interface Options extends PipelineOptions {
-
-    @Description("Kafka Bootstrap Servers")
-    @Validation.Required
-    String getBootstrapServers();
-
-    void setBootstrapServers(String value);
-
-    @Description("Kafka topic to read the input from")
-    @Validation.Required
-    String getInputTopic();
-
-    void setInputTopic(String value);
-
-    @Description(
-        "The Cloud Pub/Sub topic to publish to. "
-            + "The name should be in the format of "
-            + "projects//topics/.")
-    @Validation.Required
-    String getOutputTopic();
-
-    void setOutputTopic(String outputTopic);
-  }
-
-  /**
-   * The main entry-point for pipeline execution. This method will start the pipeline but will not
-   * wait for it's execution to finish. If blocking execution is required, use the {@link
-   * KafkaToPubSub#run(Options)} method to start the pipeline and invoke {@code
-   * result.waitUntilFinish()} on the {@link PipelineResult}.
-   *
-   * @param args The command-line args passed by the executor.
-   */
-  public static void main(String[] args) {
-    Options options = PipelineOptionsFactory.fromArgs(args).withValidation().as(Options.class);
-
-    run(options);
-  }
-
-  /**
-   * Runs the pipeline to completion with the specified options. This method does not wait until the
-   * pipeline is finished before returning. Invoke {@code result.waitUntilFinish()} on the result
-   * object to block until the pipeline is finished running if blocking programmatic execution is
-   * required.
-   *
-   * @param options The execution options.
-   * @return The pipeline result.
-   */
-  public static PipelineResult run(Options options) {
-
-    Pipeline pipeline = Pipeline.create(options);
-
-    // Register the coder for pipeline
-
-    /*
-     * Steps:
-     *  1) Read messages in from Kafka
-     *  3) Write successful records to PubSub
-     */
-    pipeline
-        .apply(
-            "ReadFromKafka",
-            KafkaIO.read()
-                .withBootstrapServers(options.getBootstrapServers())
-                .withTopic(options.getInputTopic())
-                .withKeyDeserializer(StringDeserializer.class)
-                .withValueDeserializer(StringDeserializer.class)
-                // NumSplits is hard-coded to 1 for single-partition use cases (e.g., Debezium
-                // Change Data Capture). Once Dataflow dynamic templates are available, this can
-                // be deprecated.
-                .withoutMetadata())
-        .apply(Values.create())
-        .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic()));
-
-    return pipeline.run();
-  }
-}
diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java
new file mode 100644
index 000000000000..f8c73ac7e875
--- /dev/null
+++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java
@@ -0,0 +1,185 @@
+/*
+ * 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.templates;
+
+import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import org.apache.beam.sdk.Pipeline;
+import org.apache.beam.sdk.PipelineResult;
+import org.apache.beam.sdk.coders.NullableCoder;
+import org.apache.beam.sdk.coders.StringUtf8Coder;
+import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO;
+import org.apache.beam.sdk.io.kafka.KafkaIO;
+import org.apache.beam.sdk.options.PipelineOptionsFactory;
+import org.apache.beam.sdk.transforms.Values;
+import org.apache.beam.templates.options.KafkaToPubsubOptions;
+import org.apache.kafka.common.serialization.StringDeserializer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class KafkaToPubsub {
+
+  /**
+   * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format
+   * from Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap
+   * servers are specified by the user as template parameters.
+   *
+   * 

Pipeline Requirements + * + *

    + *
  • Kafka Bootstrap Server(s). + *
  • Kafka Topic(s) exists. + *
  • The PubSub output topic exists. + *
+ * + *

Example Usage + * + *

+   * # Set the pipeline vars
+   * PROJECT=ID_OF_MY_PROJECT
+   * BUCKET_NAME=MY_BUCKET
+   *
+   * # Set containerization vars
+   * IMAGE_NAME=my-image-name
+   * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
+   * BASE_CONTAINER_IMAGE=my-base-container-image
+   * BASE_CONTAINER_IMAGE_VERSION=my-base-container-image-version
+   * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json"
+   * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
+   *
+   * # Go to the beam folder
+   * cd /path/to/beam
+   *
+   * # Create bucket in the cloud storage
+   * gsutil mb gs://${BUCKET_NAME}
+   *
+   * FLEX TEMPLATE
+   * # Assemble uber-jar
+   * ./gradlew -p examples/java clean shadowJar
+   *
+   * # Go to the template folder
+   *
+   * cd /path/to/beam/templates/kafka-to-pubsub
+   *
+   * # Build the flex template
+   * gcloud dataflow flex-template build $TEMPLATE_PATH \
+   *       --image-gcr-path "{$TARGET_GCR_IMAGE}" \
+   *       --sdk-language "JAVA" \
+   *       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
+   *       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
+   *       --jar "build/libs/beam-examples-java-2.25.0-SNAPSHOT-all.jar" \
+   *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
+   *
+   * # Execute template:
+   * API_ROOT_URL="https://dataflow.googleapis.com"
+   * TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch"
+   * JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`"
+   *
+   * time curl -X POST -H "Content-Type: application/json"     \
+   *     -H "Authorization: Bearer $(gcloud auth print-access-token)" \
+   *     "${TEMPLATES_LAUNCH_API}"`
+   *     `"?validateOnly=false"`
+   *     `"&dynamicTemplate.gcsPath=${BUCKET_NAME}/path/to/image-spec"`
+   *     `"&dynamicTemplate.stagingLocation=${BUCKET_NAME}/staging" \
+   *     -d '
+   *      {
+   *       "jobName":"'$JOB_NAME'",
+   *       "parameters": {
+   *           "bootstrapServers":"broker_1:9092,broker_2:9092",
+   *           "inputTopics":"topic1,topic2",
+   *           "outputDirectory":"'$BUCKET_NAME/path/to/output-location'",
+   *           "outputFileFormat":"text",
+   *           "outputFilenamePrefix":"output",
+   *           "windowDuration":"5m",
+   *           "numShards":"5"
+   *        }
+   *       }
+   *      '
+   * 
+ */ + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubsub.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + KafkaToPubsubOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(KafkaToPubsubOptions.class); + + run(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) { + + List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); + + checkArgument( + topicsList.size() > 0 && topicsList.get(0).length() > 0, + "inputTopics cannot be an empty string."); + + List bootstrapServersList = + new ArrayList<>(Arrays.asList(options.getBootstrapServers().split(","))); + + checkArgument( + 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() + + " input topics: " + + options.getInputTopics() + + " output pubsub topic: " + + options.getOutputTopic()); + + /* + * Steps: + * 1) Read messages in from Kafka + * 3) Write successful records to PubSub + */ + pipeline + .apply( + "ReadFromKafka", + KafkaIO.read() + .withBootstrapServers(options.getBootstrapServers()) + .withTopics(topicsList) + .withKeyDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withValueDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withoutMetadata()) + .apply(Values.create()) + .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); + + return pipeline.run(); + } +} diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java new file mode 100644 index 000000000000..30e42225d99f --- /dev/null +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -0,0 +1,45 @@ +/* + * 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.templates.options; + +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +public interface KafkaToPubsubOptions extends PipelineOptions { + @Description("Kafka Bootstrap Servers") + @Validation.Required + String getBootstrapServers(); + + void setBootstrapServers(String value); + + @Description("Kafka topic to read the input from") + @Validation.Required + String getInputTopics(); + + void setInputTopic(String value); + + @Description( + "The Cloud Pub/Sub topic to publish to. " + + "The name should be in the format of " + + "projects//topics/.") + @Validation.Required + String getOutputTopic(); + + void setOutputTopic(String outputTopic); +} From 4ab8e58a8b96ac3d5ce4ee8189b2268a103829ce Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 17:40:31 +0300 Subject: [PATCH 06/73] Added package-info.java to new packages --- .../beam/templates/options/package-info.java | 20 +++++++++++++++++++ .../apache/beam/templates/package-info.java | 20 +++++++++++++++++++ 2 files changed, 40 insertions(+) create mode 100644 templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java create mode 100644 templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java new file mode 100644 index 000000000000..5c9fa57f8ed2 --- /dev/null +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template for Google Cloud Dataflow. */ +package org.apache.beam.templates.options; diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java new file mode 100644 index 000000000000..4e7b8ca2b97b --- /dev/null +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template for Google Cloud Dataflow. */ +package org.apache.beam.templates; From aeb64114430f20a22abd671a698400d1b012f6e9 Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 18:02:53 +0300 Subject: [PATCH 07/73] Reverted build.gradle to master branch state --- examples/java/build.gradle | 20 ++++++++++---------- 1 file changed, 10 insertions(+), 10 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 10bfd4a129a0..eaad35f85937 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -20,10 +20,10 @@ import groovy.json.JsonOutput plugins { id 'org.apache.beam.module' } applyJavaNature( - exportJavadoc: false, - enableChecker: false, - ignoreRawtypeErrors: true, - automaticModuleName: 'org.apache.beam.examples') + exportJavadoc: false, + enableChecker: false, + ignoreRawtypeErrors: true, + automaticModuleName: 'org.apache.beam.examples') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() @@ -97,9 +97,9 @@ dependencies { * of integration tests for WordCount and WindowedWordCount. */ def preCommitRunnerClass = [ - directRunner: "org.apache.beam.runners.direct.DirectRunner", - flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", - sparkRunner: "org.apache.beam.runners.spark.TestSparkRunner", + directRunner: "org.apache.beam.runners.direct.DirectRunner", + flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", + sparkRunner: "org.apache.beam.runners.spark.TestSparkRunner", ] def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' @@ -107,9 +107,9 @@ def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for- for (String runner : preCommitRunners) { tasks.create(name: runner + "PreCommit", type: Test) { def preCommitBeamTestPipelineOptions = [ - "--project=${gcpProject}", - "--tempRoot=${gcsTempRoot}", - "--runner=" + preCommitRunnerClass[runner], + "--project=${gcpProject}", + "--tempRoot=${gcsTempRoot}", + "--runner=" + preCommitRunnerClass[runner], ] classpath = configurations."${runner}PreCommit" include "**/WordCountIT.class" From 61bf9921454afa9923c438d68437eeb1af6949e9 Mon Sep 17 00:00:00 2001 From: "ilya.kozyrev" Date: Fri, 2 Oct 2020 18:11:46 +0300 Subject: [PATCH 08/73] fixed JAVADOC and metadata --- .../apache/beam/templates/KafkaToPubsub.java | 17 ++++++----------- .../resources/kafka_to_pubsub_metadata.json | 2 +- 2 files changed, 7 insertions(+), 12 deletions(-) diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index f8c73ac7e875..f9c459021b60 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -54,8 +54,8 @@ public class KafkaToPubsub { * *
    * # Set the pipeline vars
-   * PROJECT=ID_OF_MY_PROJECT
-   * BUCKET_NAME=MY_BUCKET
+   * PROJECT=id-of-my-project
+   * BUCKET_NAME=my-bucket
    *
    * # Set containerization vars
    * IMAGE_NAME=my-image-name
@@ -73,11 +73,10 @@ public class KafkaToPubsub {
    *
    * FLEX TEMPLATE
    * # Assemble uber-jar
-   * ./gradlew -p examples/java clean shadowJar
+   * ./gradlew -p templates/kafka-to-pubsub clean shadowJar
    *
    * # Go to the template folder
-   *
-   * cd /path/to/beam/templates/kafka-to-pubsub
+   *cd /path/to/beam/templates/kafka-to-pubsub
    *
    * # Build the flex template
    * gcloud dataflow flex-template build $TEMPLATE_PATH \
@@ -85,7 +84,7 @@ public class KafkaToPubsub {
    *       --sdk-language "JAVA" \
    *       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
    *       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
-   *       --jar "build/libs/beam-examples-java-2.25.0-SNAPSHOT-all.jar" \
+   *       --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \
    *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
    *
    * # Execute template:
@@ -105,11 +104,7 @@ public class KafkaToPubsub {
    *       "parameters": {
    *           "bootstrapServers":"broker_1:9092,broker_2:9092",
    *           "inputTopics":"topic1,topic2",
-   *           "outputDirectory":"'$BUCKET_NAME/path/to/output-location'",
-   *           "outputFileFormat":"text",
-   *           "outputFilenamePrefix":"output",
-   *           "windowDuration":"5m",
-   *           "numShards":"5"
+   *           "outputTopic":"projects/${PROJECT}/topics/your-topic-name"
    *        }
    *       }
    *      '
diff --git a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json
index 84352fad6cf8..7bf4343f4496 100644
--- a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json
+++ b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json
@@ -24,4 +24,4 @@
       "paramType": "TEXT"
     }
   ]
-}
\ No newline at end of file
+}

From 015e6e6913656d3f95a423bdee49d3731dc40826 Mon Sep 17 00:00:00 2001
From: Artur Khanin 
Date: Mon, 5 Oct 2020 16:55:53 +0300
Subject: [PATCH 09/73] Added the Read Me section with a step-by-step guide

---
 templates/README.md                           | 146 ++++++++++++++++++
 .../apache/beam/templates/KafkaToPubsub.java  |  14 +-
 2 files changed, 153 insertions(+), 7 deletions(-)
 create mode 100644 templates/README.md

diff --git a/templates/README.md b/templates/README.md
new file mode 100644
index 000000000000..f76296ae4529
--- /dev/null
+++ b/templates/README.md
@@ -0,0 +1,146 @@
+# Dataflow Kafka to Pub/Sub Flex template
+
+This directory contains the template that creates the Dataflow pipeline 
+to read data from a single topic or multiple topics from 
+[Apache Kafka](https://kafka.apache.org/) and write it into a single topic 
+in [Google Pub/Sub](https://cloud.google.com/pubsub).
+
+## Requirements
+
+- Java 11
+- Kafka Bootstrap Server(s) up and running
+- Kafka Topic(s) exists
+- The PubSub output topic exists
+
+## Getting Started
+
+This section describes the preparation steps that need to be done before 
+the template build and execution.
+
+### Setting Environment Variables
+
+Pipeline variables:
+
+```
+PROJECT=id-of-my-project
+BUCKET_NAME=my-bucket
+REGION=my-region
+```
+
+Containerization variables:
+
+```
+IMAGE_NAME=my-image-name
+TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
+BASE_CONTAINER_IMAGE=my-base-container-image
+BASE_CONTAINER_IMAGE_VERSION=my-base-container-image-version
+TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json"
+TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
+```
+
+### Bucket Creation
+
+The Dataflow Flex templates has to store its metadata in a bucket in 
+Google Cloud Storage, so it can be executed from the Google Cloud Platform.
+Create the bucket in Google Cloud Storage if it doesn't exist yet:
+
+```
+gsutil mb gs://${BUCKET_NAME}
+```
+
+## The Dataflow Flex Template
+
+Flex Templates package the pipeline as a Docker image and stage these images 
+on your project's Container Registry.
+
+### Assembling the Uber-JAR
+
+The Dataflow Flex templates require your Java project to be built into 
+an Uber JAR file.
+
+Go to the Beam folder:
+
+```
+cd /path/to/beam
+```
+
+In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) 
+is used. It creates the `shadowJar` task that builds the Uber JAR:
+
+```
+./gradlew -p templates/kafka-to-pubsub clean shadowJar
+```
+
+ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains 
+both target package *and* all its dependencies.
+
+The result of the `shadowJar` task execution is a `.jar` file that is generated 
+under the `build/libs/` folder in kafka-to-pubsub directory.
+
+### Creating the Flex Template
+
+To execute the template you need to create the template spec file containing all
+the necessary information to run the job. This template already has such [metadata
+file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources.
+
+Go to the template folder:
+
+```
+cd /path/to/beam/templates/kafka-to-pubsub
+```
+
+Build the Flex Template:
+
+```
+gcloud dataflow flex-template build ${TEMPLATE_PATH} \
+       --image-gcr-path "{$TARGET_GCR_IMAGE}" \
+       --sdk-language "JAVA" \
+       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
+       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
+       --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \
+       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
+```
+
+### Running the Pipeline
+
+To deploy the pipeline, you should refer to the template file and pass the 
+[parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) 
+required by the pipeline.
+
+You can do this in 3 different ways:
+1. Using [Dataflow Google Cloud Console](https://console.cloud.google.com/dataflow/jobs)
+
+2. Using `gcloud` CLI tool
+    ```
+    gcloud dataflow flex-template run "kafka-to-pubsub-`date +%Y%m%d-%H%M%S`" \
+        --template-file-gcs-location "${TEMPLATE_PATH}" \
+        --parameters bootstrapServers="broker_1:9092,broker_2:9092" \
+        --parameters inputTopics="topic1,topic2" \
+        --parameters outputTopic="projects/${PROJECT}/topics/your-topic-name" \
+        --region "${REGION}"
+    ```
+3. With a REST API request
+    ```
+    API_ROOT_URL="https://dataflow.googleapis.com"
+    TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch"
+    JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`"
+    
+    time curl -X POST -H "Content-Type: application/json" \
+        -H "Authorization: Bearer $(gcloud auth print-access-token)" \
+        "${TEMPLATES_LAUNCH_API}"`
+        `"?validateOnly=false"`
+        `"&dynamicTemplate.gcsPath=${BUCKET_NAME}/path/to/image-spec"`
+        `"&dynamicTemplate.stagingLocation=${BUCKET_NAME}/staging" \
+        -d '
+         {
+          "jobName":"${JOB_NAME}",
+          "parameters": {
+              "bootstrapServers":"broker_1:9092,broker_2:9092",
+              "inputTopics":"topic1,topic2",
+              "outputTopic":"projects/${PROJECT}/topics/your-topic-name"
+          }
+         }
+        '
+    ```
+
+
diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java
index f9c459021b60..75069db02880 100644
--- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java
+++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java
@@ -39,7 +39,7 @@ public class KafkaToPubsub {
 
   /**
    * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format
-   * from Kafka, and outputs the resulting records to PubSub. Input topic, output topic, Bootstrap
+   * from Kafka, and outputs the resulting records to PubSub. Input topics, output topic, Bootstrap
    * servers are specified by the user as template parameters.
    *
    * 

Pipeline Requirements @@ -65,21 +65,21 @@ public class KafkaToPubsub { * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} * - * # Go to the beam folder - * cd /path/to/beam - * * # Create bucket in the cloud storage * gsutil mb gs://${BUCKET_NAME} * + * # Go to the beam folder + * cd /path/to/beam + * * FLEX TEMPLATE * # Assemble uber-jar * ./gradlew -p templates/kafka-to-pubsub clean shadowJar * * # Go to the template folder - *cd /path/to/beam/templates/kafka-to-pubsub + * cd /path/to/beam/templates/kafka-to-pubsub * * # Build the flex template - * gcloud dataflow flex-template build $TEMPLATE_PATH \ + * gcloud dataflow flex-template build ${TEMPLATE_PATH} \ * --image-gcr-path "{$TARGET_GCR_IMAGE}" \ * --sdk-language "JAVA" \ * --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ @@ -100,7 +100,7 @@ public class KafkaToPubsub { * `"&dynamicTemplate.stagingLocation=${BUCKET_NAME}/staging" \ * -d ' * { - * "jobName":"'$JOB_NAME'", + * "jobName":"${JOB_NAME}", * "parameters": { * "bootstrapServers":"broker_1:9092,broker_2:9092", * "inputTopics":"topic1,topic2", From 129f4666adf36c453ff56f69b60ae8f13456b38e Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Mon, 5 Oct 2020 09:52:02 -0700 Subject: [PATCH 10/73] Update README.md --- templates/README.md | 20 +++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/templates/README.md b/templates/README.md index f76296ae4529..1420355ed263 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,20 +1,22 @@ -# Dataflow Kafka to Pub/Sub Flex template +# Dataflow Kafka to Pub/Sub Flex Template -This directory contains the template that creates the Dataflow pipeline +This directory contains a Flex template that creates a Dataflow pipeline to read data from a single topic or multiple topics from -[Apache Kafka](https://kafka.apache.org/) and write it into a single topic +[Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). +This template supports serializable string formats, such as JSON. + ## Requirements - Java 11 - Kafka Bootstrap Server(s) up and running -- Kafka Topic(s) exists -- The PubSub output topic exists +- Source Kafka Topic(s) +- Destination PubSub output topic created ## Getting Started -This section describes the preparation steps that need to be done before +This section describes the preparation steps that are prerequisite for the template build and execution. ### Setting Environment Variables @@ -40,7 +42,7 @@ TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ### Bucket Creation -The Dataflow Flex templates has to store its metadata in a bucket in +The Dataflow Flex template has to store its metadata in a bucket in Google Cloud Storage, so it can be executed from the Google Cloud Platform. Create the bucket in Google Cloud Storage if it doesn't exist yet: @@ -58,7 +60,7 @@ on your project's Container Registry. The Dataflow Flex templates require your Java project to be built into an Uber JAR file. -Go to the Beam folder: +Navigate to the Beam folder: ``` cd /path/to/beam @@ -83,7 +85,7 @@ To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has such [metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. -Go to the template folder: +Navigate to the template folder: ``` cd /path/to/beam/templates/kafka-to-pubsub From 6db17adc37596b202332e38ddc2f8b61df482335 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 7 Oct 2020 11:32:33 +0300 Subject: [PATCH 11/73] Readme fixes regarding comments --- templates/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/templates/README.md b/templates/README.md index 1420355ed263..eaac92fc088d 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,6 +1,6 @@ -# Dataflow Kafka to Pub/Sub Flex Template +# Dataflow Flex Template to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains a Flex template that creates a Dataflow pipeline +This directory contains a Dataflow Flex Template that creates a pipeline to read data from a single topic or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). @@ -11,13 +11,13 @@ This template supports serializable string formats, such as JSON. - Java 11 - Kafka Bootstrap Server(s) up and running -- Source Kafka Topic(s) -- Destination PubSub output topic created +- Existing Kafka topic(s) +- An existing Pub/Sub destination output topic ## Getting Started -This section describes the preparation steps that are prerequisite for -the template build and execution. +This section describes what is needed in order to get the template +up and running. ### Setting Environment Variables From 01e4987c5957272409dc4f4f86f359b3aac77aa5 Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Wed, 7 Oct 2020 06:56:24 -0700 Subject: [PATCH 12/73] Update README.md --- templates/README.md | 39 +++++++++++++++++++-------------------- 1 file changed, 19 insertions(+), 20 deletions(-) diff --git a/templates/README.md b/templates/README.md index eaac92fc088d..7e6bcd805d76 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,7 +1,7 @@ # Dataflow Flex Template to ingest data from Apache Kafka to Google Cloud Pub/Sub This directory contains a Dataflow Flex Template that creates a pipeline -to read data from a single topic or multiple topics from +to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). @@ -11,17 +11,16 @@ This template supports serializable string formats, such as JSON. - Java 11 - Kafka Bootstrap Server(s) up and running -- Existing Kafka topic(s) +- Existing source Kafka topic(s) - An existing Pub/Sub destination output topic ## Getting Started -This section describes what is needed in order to get the template -up and running. +This section describes steps to set up the environment, build [Apache Kafka](https://kafka.apache.org/) to [Google Pub/Sub](https://cloud.google.com/pubsub) Flex Dataflow template, and create a Dataflow job to ingest data usign the template. -### Setting Environment Variables +### Setting Up Project Environment -Pipeline variables: +#### Pipeline variables: ``` PROJECT=id-of-my-project @@ -29,7 +28,17 @@ BUCKET_NAME=my-bucket REGION=my-region ``` -Containerization variables: +#### Template Metadata Storage Bucket Creation + +The Dataflow Flex template has to store its metadata in a bucket in +Google Cloud Storage, so it can be executed from the Google Cloud Platform. +Create the bucket in Google Cloud Storage if it doesn't exist yet: + +``` +gsutil mb gs://${BUCKET_NAME} +``` + +#### Containerization variables: ``` IMAGE_NAME=my-image-name @@ -40,17 +49,7 @@ TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ``` -### Bucket Creation - -The Dataflow Flex template has to store its metadata in a bucket in -Google Cloud Storage, so it can be executed from the Google Cloud Platform. -Create the bucket in Google Cloud Storage if it doesn't exist yet: - -``` -gsutil mb gs://${BUCKET_NAME} -``` - -## The Dataflow Flex Template +## Build Apache Kafka to Google Pub/Sub Flex Dataflow Template Flex Templates package the pipeline as a Docker image and stage these images on your project's Container Registry. @@ -82,7 +81,7 @@ under the `build/libs/` folder in kafka-to-pubsub directory. ### Creating the Flex Template To execute the template you need to create the template spec file containing all -the necessary information to run the job. This template already has such [metadata +the necessary information to run the job. This template already has the following [metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: @@ -103,7 +102,7 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" ``` -### Running the Pipeline +### Create Dataflow Job from the Apache Kafka to Google Pub/Sub Flex Dataflow Template To deploy the pipeline, you should refer to the template file and pass the [parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) From b6decbd1d617770f5e74e048306d6eb94c01ebd5 Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Wed, 7 Oct 2020 07:13:02 -0700 Subject: [PATCH 13/73] Update README.md --- templates/README.md | 17 ++++++++++------- 1 file changed, 10 insertions(+), 7 deletions(-) diff --git a/templates/README.md b/templates/README.md index 7e6bcd805d76..8017b014fff7 100644 --- a/templates/README.md +++ b/templates/README.md @@ -16,7 +16,10 @@ This template supports serializable string formats, such as JSON. ## Getting Started -This section describes steps to set up the environment, build [Apache Kafka](https://kafka.apache.org/) to [Google Pub/Sub](https://cloud.google.com/pubsub) Flex Dataflow template, and create a Dataflow job to ingest data usign the template. +This section describes what is needed to get the template up and rrunning. +- Set up the environment +- Build Apache Kafka to Google Pub/Sub Dataflow Flex template +- Create a Dataflow job to ingest data usign the template. ### Setting Up Project Environment @@ -31,7 +34,7 @@ REGION=my-region #### Template Metadata Storage Bucket Creation The Dataflow Flex template has to store its metadata in a bucket in -Google Cloud Storage, so it can be executed from the Google Cloud Platform. +[Google Cloud Storage](https://cloud.google.com/storage), so it can be executed from the Google Cloud Platform. Create the bucket in Google Cloud Storage if it doesn't exist yet: ``` @@ -51,8 +54,8 @@ TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ## Build Apache Kafka to Google Pub/Sub Flex Dataflow Template -Flex Templates package the pipeline as a Docker image and stage these images -on your project's Container Registry. +Dataflow Flex templates package the pipeline as a Docker image and stage these images +on your project's [Container Registry](https://cloud.google.com/container-registry). ### Assembling the Uber-JAR @@ -78,7 +81,7 @@ both target package *and* all its dependencies. The result of the `shadowJar` task execution is a `.jar` file that is generated under the `build/libs/` folder in kafka-to-pubsub directory. -### Creating the Flex Template +### Creating the Dataflow Flex Template To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has the following [metadata @@ -90,7 +93,7 @@ Navigate to the template folder: cd /path/to/beam/templates/kafka-to-pubsub ``` -Build the Flex Template: +Build the Dataflow Flex template: ``` gcloud dataflow flex-template build ${TEMPLATE_PATH} \ @@ -102,7 +105,7 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" ``` -### Create Dataflow Job from the Apache Kafka to Google Pub/Sub Flex Dataflow Template +### Create Dataflow Job Using the Apache Kafka to Google Pub/Sub Dataflow Flex Template To deploy the pipeline, you should refer to the template file and pass the [parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) From 8e41d136f1cc94eb2442608fd7117b8d7972fd2c Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 7 Oct 2020 17:24:01 +0300 Subject: [PATCH 14/73] Fixed typos in README.md --- templates/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/templates/README.md b/templates/README.md index 8017b014fff7..7e9f5552f107 100644 --- a/templates/README.md +++ b/templates/README.md @@ -16,10 +16,10 @@ This template supports serializable string formats, such as JSON. ## Getting Started -This section describes what is needed to get the template up and rrunning. +This section describes what is needed to get the template up and running. - Set up the environment -- Build Apache Kafka to Google Pub/Sub Dataflow Flex template -- Create a Dataflow job to ingest data usign the template. +- Build Apache Kafka to Google Pub/Sub Dataflow Flex Template +- Create a Dataflow job to ingest data using the template. ### Setting Up Project Environment @@ -54,12 +54,12 @@ TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ## Build Apache Kafka to Google Pub/Sub Flex Dataflow Template -Dataflow Flex templates package the pipeline as a Docker image and stage these images +Dataflow Flex Templates package the pipeline as a Docker image and stage these images on your project's [Container Registry](https://cloud.google.com/container-registry). ### Assembling the Uber-JAR -The Dataflow Flex templates require your Java project to be built into +The Dataflow Flex Templates require your Java project to be built into an Uber JAR file. Navigate to the Beam folder: @@ -93,7 +93,7 @@ Navigate to the template folder: cd /path/to/beam/templates/kafka-to-pubsub ``` -Build the Dataflow Flex template: +Build the Dataflow Flex Template: ``` gcloud dataflow flex-template build ${TEMPLATE_PATH} \ From 369a2058ac462da9bb2658bc756a85b66b9d1f28 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Wed, 7 Oct 2020 18:32:31 +0300 Subject: [PATCH 15/73] refactored README.md added case to run template locally --- templates/README.md | 80 +++++++++++++++++++++++++++------------------ 1 file changed, 48 insertions(+), 32 deletions(-) diff --git a/templates/README.md b/templates/README.md index 7e9f5552f107..b87723db2351 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,6 +1,6 @@ -# Dataflow Flex Template to ingest data from Apache Kafka to Google Cloud Pub/Sub +# Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains a Dataflow Flex Template that creates a pipeline +This directory contains an Apache Beam Template that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). @@ -17,9 +17,51 @@ This template supports serializable string formats, such as JSON. ## Getting Started This section describes what is needed to get the template up and running. -- Set up the environment -- Build Apache Kafka to Google Pub/Sub Dataflow Flex Template -- Create a Dataflow job to ingest data using the template. +- Assembling the Uber-JAR +- Local execution +- Google Dataflow Template + - Set up the environment + - Creating the Dataflow Flex Template + - Create a Dataflow job to ingest data using the template. + +## Assembling the Uber-JAR + +To run this template correctly needs to your Java project to be built into +an Uber JAR file. + +Navigate to the Beam folder: + +``` +cd /path/to/beam +``` + +In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) +is used. It creates the `shadowJar` task that builds the Uber JAR: + +``` +./gradlew -p templates/kafka-to-pubsub clean shadowJar +``` + +ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains +both target package *and* all its dependencies. + +The result of the `shadowJar` task execution is a `.jar` file that is generated +under the `build/libs/` folder in kafka-to-pubsub directory. + +## Local execution +To execute this pipeline locally, specify the parameters: Kafka Bootstrap servers, Kafka input topic, Pub/Sub output topic with the form: +```bash +--bootstrapServers=host:port \ +--inputTopic=your-input-topic \ +--outputTopic=projects/your-project-id/topics/your-topic-pame +``` +To change the runner, specify: + +```--runner=YOUR_SELECTED_RUNNER``` + +See examples/java/README.md for instructions about how to configure different runners. + +## Google Dataflow Template ### Setting Up Project Environment @@ -52,37 +94,11 @@ TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ``` -## Build Apache Kafka to Google Pub/Sub Flex Dataflow Template +### Creating the Dataflow Flex Template Dataflow Flex Templates package the pipeline as a Docker image and stage these images on your project's [Container Registry](https://cloud.google.com/container-registry). -### Assembling the Uber-JAR - -The Dataflow Flex Templates require your Java project to be built into -an Uber JAR file. - -Navigate to the Beam folder: - -``` -cd /path/to/beam -``` - -In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) -is used. It creates the `shadowJar` task that builds the Uber JAR: - -``` -./gradlew -p templates/kafka-to-pubsub clean shadowJar -``` - -ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains -both target package *and* all its dependencies. - -The result of the `shadowJar` task execution is a `.jar` file that is generated -under the `build/libs/` folder in kafka-to-pubsub directory. - -### Creating the Dataflow Flex Template - To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has the following [metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. From 01db016e4bafb1aa1e525b02e44e2923d82488f6 Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Wed, 7 Oct 2020 08:56:03 -0700 Subject: [PATCH 16/73] Update README.md --- templates/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/templates/README.md b/templates/README.md index b87723db2351..2ff0deb73983 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,6 +1,6 @@ # Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains an Apache Beam Template that creates a pipeline +This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). @@ -26,7 +26,7 @@ This section describes what is needed to get the template up and running. ## Assembling the Uber-JAR -To run this template correctly needs to your Java project to be built into +To run this template the template Java project should be built into an Uber JAR file. Navigate to the Beam folder: From 5821bdff66137df7a9978419d384e3b18079b6a6 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 8 Oct 2020 15:46:41 +0300 Subject: [PATCH 17/73] fix build script for dataflow in README.md --- templates/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/templates/README.md b/templates/README.md index 2ff0deb73983..0e035037c2b7 100644 --- a/templates/README.md +++ b/templates/README.md @@ -113,7 +113,7 @@ Build the Dataflow Flex Template: ``` gcloud dataflow flex-template build ${TEMPLATE_PATH} \ - --image-gcr-path "{$TARGET_GCR_IMAGE}" \ + --image-gcr-path {$TARGET_GCR_IMAGE} \ --sdk-language "JAVA" \ --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ From 98de91f880cbf01585a24de12c6c459af1ee7380 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 9 Oct 2020 11:19:40 +0300 Subject: [PATCH 18/73] Added unit test and fixed metadata file --- templates/README.md | 4 +- templates/kafka-to-pubsub/build.gradle | 3 +- .../apache/beam/templates/KafkaToPubsub.java | 28 ++++++---- .../options/KafkaToPubsubOptions.java | 4 +- .../resources/kafka_to_pubsub_metadata.json | 8 +-- .../KafkaToPubsubTest.java | 53 +++++++++++++++++++ 6 files changed, 79 insertions(+), 21 deletions(-) create mode 100644 templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java diff --git a/templates/README.md b/templates/README.md index 0e035037c2b7..c8d782af841e 100644 --- a/templates/README.md +++ b/templates/README.md @@ -89,9 +89,7 @@ gsutil mb gs://${BUCKET_NAME} IMAGE_NAME=my-image-name TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} BASE_CONTAINER_IMAGE=my-base-container-image -BASE_CONTAINER_IMAGE_VERSION=my-base-container-image-version TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" -TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} ``` ### Creating the Dataflow Flex Template @@ -113,7 +111,7 @@ Build the Dataflow Flex Template: ``` gcloud dataflow flex-template build ${TEMPLATE_PATH} \ - --image-gcr-path {$TARGET_GCR_IMAGE} \ + --image-gcr-path ${TARGET_GCR_IMAGE} \ --sdk-language "JAVA" \ --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ diff --git a/templates/kafka-to-pubsub/build.gradle b/templates/kafka-to-pubsub/build.gradle index d5e6640e7971..c5cada1fde08 100644 --- a/templates/kafka-to-pubsub/build.gradle +++ b/templates/kafka-to-pubsub/build.gradle @@ -21,13 +21,14 @@ repositories { } dependencies { - testCompile group: 'junit', name: 'junit', version: '4.12' compile library.java.kafka_clients compile project(path: ":sdks:java:core", 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") compile project(":runners:google-cloud-dataflow-java") + runtime project(path: ":runners:direct-java", configuration: "shadow") + testCompile group: 'junit', name: 'junit', version: '4.12' } def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 75069db02880..bc98aa49e74d 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -29,7 +29,11 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.kafka.common.serialization.StringDeserializer; import org.slf4j.Logger; @@ -61,9 +65,7 @@ public class KafkaToPubsub { * IMAGE_NAME=my-image-name * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} * BASE_CONTAINER_IMAGE=my-base-container-image - * BASE_CONTAINER_IMAGE_VERSION=my-base-container-image-version * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" - * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} * * # Create bucket in the cloud storage * gsutil mb gs://${BUCKET_NAME} @@ -80,7 +82,7 @@ public class KafkaToPubsub { * * # Build the flex template * gcloud dataflow flex-template build ${TEMPLATE_PATH} \ - * --image-gcr-path "{$TARGET_GCR_IMAGE}" \ + * --image-gcr-path "${TARGET_GCR_IMAGE}" \ * --sdk-language "JAVA" \ * --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ * --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ @@ -126,6 +128,17 @@ public static void main(String[] args) { run(options); } + public static PTransform>> readFromKafka(String bootstrapServers, List topicsList) { + return KafkaIO.read() + .withBootstrapServers(bootstrapServers) + .withTopics(topicsList) + .withKeyDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withValueDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withoutMetadata(); + } + /** * Runs a pipeline which reads message from Kafka and writes it to GCS. * @@ -164,14 +177,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { pipeline .apply( "ReadFromKafka", - KafkaIO.read() - .withBootstrapServers(options.getBootstrapServers()) - .withTopics(topicsList) - .withKeyDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withValueDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withoutMetadata()) + readFromKafka(options.getBootstrapServers(), topicsList)) .apply(Values.create()) .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 30e42225d99f..d2675562bcf7 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -28,11 +28,11 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setBootstrapServers(String value); - @Description("Kafka topic to read the input from") + @Description("Kafka topics to read the input from") @Validation.Required String getInputTopics(); - void setInputTopic(String value); + void setInputTopics(String value); @Description( "The Cloud Pub/Sub topic to publish to. " diff --git a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index 7bf4343f4496..9823fc15da39 100644 --- a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -5,14 +5,14 @@ { "name": "bootstrapServers", "label": "Kafka Bootstrap Servers", - "helpText": "Comma separated kafka bootstrap servers in format ip:port", + "helpText": "Comma separated Kafka bootstrap servers in format ip:port", "regexes": [], "paramType": "TEXT" }, { - "name": "inputTopic", - "label": "Kafka input topic", - "helpText": "Kafka topic to read data", + "name": "inputTopics", + "label": "Kafka input topics", + "helpText": "Comma separated Kafka topics to read data", "regexes": [], "paramType": "TEXT" }, diff --git a/templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java b/templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java new file mode 100644 index 000000000000..f276b2d8d89a --- /dev/null +++ b/templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java @@ -0,0 +1,53 @@ +/* + * 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.templates; + +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.testing.TestPipeline; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; +import org.junit.runner.RunWith; +import org.junit.runners.JUnit4; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; + +/** Test of KafkaToPubsub. */ +@RunWith(JUnit4.class) +public class KafkaToPubsubTest { + + @Rule + public final transient TestPipeline pipeline = TestPipeline.create(); + + @Rule + public ExpectedException thrown = ExpectedException.none(); + + @Test + public void testKafkaReadingFailsWrongBootstrapServer() { + final String bootstrapServers = "some-server:9092"; + final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); + + pipeline.apply( + KafkaToPubsub.readFromKafka(bootstrapServers, topicsList)); + thrown.expect(Pipeline.PipelineExecutionException.class); + thrown.expectMessage("Failed to construct kafka consumer"); + pipeline.run(); + } +} From fc1ed9adb283ad69441c74cb1858dcc7e81a5668 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 9 Oct 2020 12:21:25 +0300 Subject: [PATCH 19/73] Added Licenses and style fixes --- templates/README.md | 19 +++++++++++ templates/build.gradle | 18 +++++++++++ templates/kafka-to-pubsub/build.gradle | 18 +++++++++++ .../apache/beam/templates/KafkaToPubsub.java | 21 ++++++------ .../beam/templates}/KafkaToPubsubTest.java | 32 ++++++++----------- 5 files changed, 79 insertions(+), 29 deletions(-) rename templates/kafka-to-pubsub/src/test/java/{org.apache.beam.templates => org/apache/beam/templates}/KafkaToPubsubTest.java (65%) diff --git a/templates/README.md b/templates/README.md index c8d782af841e..3d1c8af2ac04 100644 --- a/templates/README.md +++ b/templates/README.md @@ -1,3 +1,22 @@ + + # Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline diff --git a/templates/build.gradle b/templates/build.gradle index 4c49b4adf050..f6934dbe6c72 100644 --- a/templates/build.gradle +++ b/templates/build.gradle @@ -1,3 +1,21 @@ +/* + * 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. + */ + plugins { id 'java' } diff --git a/templates/kafka-to-pubsub/build.gradle b/templates/kafka-to-pubsub/build.gradle index c5cada1fde08..34823e30aa0a 100644 --- a/templates/kafka-to-pubsub/build.gradle +++ b/templates/kafka-to-pubsub/build.gradle @@ -1,3 +1,21 @@ +/* + * 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. + */ + plugins { id 'java' id 'org.apache.beam.module' diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index bc98aa49e74d..919e8d1707e4 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -128,15 +128,16 @@ public static void main(String[] args) { run(options); } - public static PTransform>> readFromKafka(String bootstrapServers, List topicsList) { + public static PTransform>> readFromKafka( + String bootstrapServers, List topicsList) { return KafkaIO.read() - .withBootstrapServers(bootstrapServers) - .withTopics(topicsList) - .withKeyDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withValueDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withoutMetadata(); + .withBootstrapServers(bootstrapServers) + .withTopics(topicsList) + .withKeyDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withValueDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withoutMetadata(); } /** @@ -175,9 +176,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { * 3) Write successful records to PubSub */ pipeline - .apply( - "ReadFromKafka", - readFromKafka(options.getBootstrapServers(), topicsList)) + .apply("ReadFromKafka", readFromKafka(options.getBootstrapServers(), topicsList)) .apply(Values.create()) .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); diff --git a/templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java b/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java similarity index 65% rename from templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java rename to templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java index f276b2d8d89a..200d1b14cd47 100644 --- a/templates/kafka-to-pubsub/src/test/java/org.apache.beam.templates/KafkaToPubsubTest.java +++ b/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java @@ -17,6 +17,9 @@ */ package org.apache.beam.templates; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.TestPipeline; import org.junit.Rule; @@ -25,29 +28,22 @@ import org.junit.runner.RunWith; import org.junit.runners.JUnit4; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; - /** Test of KafkaToPubsub. */ @RunWith(JUnit4.class) public class KafkaToPubsubTest { - @Rule - public final transient TestPipeline pipeline = TestPipeline.create(); + @Rule public final transient TestPipeline pipeline = TestPipeline.create(); - @Rule - public ExpectedException thrown = ExpectedException.none(); + @Rule public ExpectedException thrown = ExpectedException.none(); - @Test - public void testKafkaReadingFailsWrongBootstrapServer() { - final String bootstrapServers = "some-server:9092"; - final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); + @Test + public void testKafkaReadingFailsWrongBootstrapServer() { + final String bootstrapServers = "some-server:9092"; + final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); - pipeline.apply( - KafkaToPubsub.readFromKafka(bootstrapServers, topicsList)); - thrown.expect(Pipeline.PipelineExecutionException.class); - thrown.expectMessage("Failed to construct kafka consumer"); - pipeline.run(); - } + pipeline.apply(KafkaToPubsub.readFromKafka(bootstrapServers, topicsList)); + thrown.expect(Pipeline.PipelineExecutionException.class); + thrown.expectMessage("Failed to construct kafka consumer"); + pipeline.run(); + } } From 6e159c7551d135223ef2653992936e5515b88d66 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 12 Oct 2020 18:29:01 +0300 Subject: [PATCH 20/73] Added support for retrieving Kafka credentials from HashiCorp Vault secret storage with url and token --- .../apache/beam/templates/KafkaToPubsub.java | 64 ++++++++++++++++++- .../options/KafkaToPubsubOptions.java | 10 +++ .../beam/templates/KafkaToPubsubTest.java | 4 +- 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 919e8d1707e4..7bd745efe9f1 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -19,9 +19,14 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; +import com.google.gson.JsonObject; +import com.google.gson.JsonParser; +import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.coders.NullableCoder; @@ -35,6 +40,15 @@ import org.apache.beam.sdk.values.PBegin; import org.apache.beam.sdk.values.PCollection; import org.apache.beam.templates.options.KafkaToPubsubOptions; +import org.apache.http.HttpResponse; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.CloseableHttpClient; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.apache.kafka.common.security.scram.ScramMechanism; import org.apache.kafka.common.serialization.StringDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,7 +143,7 @@ public static void main(String[] args) { } public static PTransform>> readFromKafka( - String bootstrapServers, List topicsList) { + String bootstrapServers, List topicsList, Map config) { return KafkaIO.read() .withBootstrapServers(bootstrapServers) .withTopics(topicsList) @@ -137,15 +151,59 @@ public static PTransform>> readFromKafka( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) .withValueDeserializerAndCoder( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withConsumerConfigUpdates(config) .withoutMetadata(); } + public static Map configureKafka(String secretStoreUrl, String token) + throws KafkaException { + try (CloseableHttpClient client = HttpClientBuilder.create().build()) { + HttpGet request = new HttpGet(secretStoreUrl); + request.addHeader("X-Vault-Token", token); + HttpResponse response = client.execute(request); + String json = EntityUtils.toString(response.getEntity(), "UTF-8"); + + JsonObject credentials = + JsonParser.parseString(json) + .getAsJsonObject() + .get("data") + .getAsJsonObject() + .getAsJsonObject("data"); + String username = credentials.get("username").getAsString(); + String password = credentials.get("password").getAsString(); + + Map config = new HashMap<>(); + config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_256.mechanismName()); + config.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.name()); + config.put( + SaslConfigs.SASL_JAAS_CONFIG, + String.format( + "org.apache.kafka.common.security.scram.ScramLoginModule required " + + "username=\"%s\" password=\"%s\";", + username, password)); + return config; + + } catch (IOException exception) { + throw new KafkaException("Failed to retrieve credentials for Kafka client:", exception); + } + } + /** * 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) { + Map kafkaConfig = new HashMap<>(); + try { + String secretStoreUrl = options.getSecretStoreUrl(); + String token = options.getVaultToken(); + kafkaConfig.putAll(configureKafka(secretStoreUrl, token)); + } catch (NullPointerException exception) { + LOG.info( + "No information to retrieve Kafka credentials was provided. " + + "Trying to initiate unauthorized connection."); + } List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); @@ -173,10 +231,12 @@ public static PipelineResult run(KafkaToPubsubOptions options) { /* * Steps: * 1) Read messages in from Kafka + * 2) Extract values only * 3) Write successful records to PubSub */ pipeline - .apply("ReadFromKafka", readFromKafka(options.getBootstrapServers(), topicsList)) + .apply( + "ReadFromKafka", readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) .apply(Values.create()) .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index d2675562bcf7..1690ec3b28d8 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -42,4 +42,14 @@ public interface KafkaToPubsubOptions extends PipelineOptions { String getOutputTopic(); void setOutputTopic(String outputTopic); + + @Description("") + String getSecretStoreUrl(); + + void setSecretStoreUrl(String secretStoreUrl); + + @Description("") + String getVaultToken(); + + void setVaultToken(String vaultToken); } diff --git a/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java index 200d1b14cd47..89a1e36aa2b0 100644 --- a/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java +++ b/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java @@ -19,6 +19,7 @@ import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.TestPipeline; @@ -40,8 +41,9 @@ public class KafkaToPubsubTest { public void testKafkaReadingFailsWrongBootstrapServer() { final String bootstrapServers = "some-server:9092"; final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); + final HashMap config = new HashMap<>(); - pipeline.apply(KafkaToPubsub.readFromKafka(bootstrapServers, topicsList)); + pipeline.apply(KafkaToPubsub.readFromKafka(bootstrapServers, topicsList, config)); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage("Failed to construct kafka consumer"); pipeline.run(); From 523f796b5bd42eca177c3e4a170cb6c5b26bf2e2 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Tue, 13 Oct 2020 17:04:52 +0300 Subject: [PATCH 21/73] Updated README.md and metadata with parameters for Vault access; refactored Kafka configuration --- templates/README.md | 16 +++- .../apache/beam/templates/KafkaToPubsub.java | 88 ++++++++++++------- .../resources/kafka_to_pubsub_metadata.json | 16 ++++ 3 files changed, 84 insertions(+), 36 deletions(-) diff --git a/templates/README.md b/templates/README.md index 3d1c8af2ac04..a3aff1093503 100644 --- a/templates/README.md +++ b/templates/README.md @@ -22,7 +22,9 @@ This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic -in [Google Pub/Sub](https://cloud.google.com/pubsub). +in [Google Pub/Sub](https://cloud.google.com/pubsub). This template provides +availability to retrieve credentials for Apache Kafka from [HashiCorp Vault](https://www.vaultproject.io/) +in case of configured SASL/SCRAM security mechanism. This template supports serializable string formats, such as JSON. @@ -32,6 +34,7 @@ This template supports serializable string formats, such as JSON. - Kafka Bootstrap Server(s) up and running - Existing source Kafka topic(s) - An existing Pub/Sub destination output topic +- (Optional) An existing HashiCorp Vault secret storage ## Getting Started @@ -74,6 +77,11 @@ To execute this pipeline locally, specify the parameters: Kafka Bootstrap server --inputTopic=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame ``` +Optionally, specify a URL to credentials in Vault and token to access them: +```bash +--secretStoreUrl=http(s)://host:port/path/to/credentials +--vaultToken=your-token +``` To change the runner, specify: ```--runner=YOUR_SELECTED_RUNNER``` @@ -154,6 +162,8 @@ You can do this in 3 different ways: --parameters bootstrapServers="broker_1:9092,broker_2:9092" \ --parameters inputTopics="topic1,topic2" \ --parameters outputTopic="projects/${PROJECT}/topics/your-topic-name" \ + --parameters secretStoreUrl="http(s)://host:port/path/to/credentials" + --parameters vaultToken="your-token" --region "${REGION}" ``` 3. With a REST API request @@ -174,7 +184,9 @@ You can do this in 3 different ways: "parameters": { "bootstrapServers":"broker_1:9092,broker_2:9092", "inputTopics":"topic1,topic2", - "outputTopic":"projects/${PROJECT}/topics/your-topic-name" + "outputTopic":"projects/${PROJECT}/topics/your-topic-name", + "secretStoreUrl":"http(s)://host:port/path/to/credentials", + "vaultToken":"your-token" } } ' diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 7bd745efe9f1..22a6dc40de70 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -41,11 +41,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.http.HttpResponse; +import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpGet; -import org.apache.http.impl.client.CloseableHttpClient; import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.util.EntityUtils; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.security.scram.ScramMechanism; @@ -58,7 +57,10 @@ public class KafkaToPubsub { /** * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format * from Kafka, and outputs the resulting records to PubSub. Input topics, output topic, Bootstrap - * servers are specified by the user as template parameters. + * servers are specified by the user as template parameters.
+ * Kafka may be configured with SASL/SCRAM security mechanism, + * in this case a Vault secret storage with credentials should be provided. URL to credentials and Vault token + * are specified by the user as template parameters. * *

Pipeline Requirements * @@ -66,6 +68,7 @@ public class KafkaToPubsub { *

  • Kafka Bootstrap Server(s). *
  • Kafka Topic(s) exists. *
  • The PubSub output topic exists. + *
  • (Optional) An existing HashiCorp Vault secret storage * * *

    Example Usage @@ -120,7 +123,9 @@ public class KafkaToPubsub { * "parameters": { * "bootstrapServers":"broker_1:9092,broker_2:9092", * "inputTopics":"topic1,topic2", - * "outputTopic":"projects/${PROJECT}/topics/your-topic-name" + * "outputTopic":"projects/${PROJECT}/topics/your-topic-name", + * "secretStoreUrl":"http(s)://host:port/path/to/credentials", + * "vaultToken":"your-token" * } * } * ' @@ -155,37 +160,45 @@ public static PTransform>> readFromKafka( .withoutMetadata(); } + /** + * Retrieves username and password from HashiCorp Vault secret storage and configures Kafka + * consumer for authorized connection. + * + * @param secretStoreUrl url to the secret storage that contains a credentials for Kafka + * @param token Vault token to access the secret storage + * @return configuration set of parameters for Kafka + * @throws IOException throws in case of the failure to execute the request to the secret storage + */ public static Map configureKafka(String secretStoreUrl, String token) - throws KafkaException { - try (CloseableHttpClient client = HttpClientBuilder.create().build()) { - HttpGet request = new HttpGet(secretStoreUrl); - request.addHeader("X-Vault-Token", token); - HttpResponse response = client.execute(request); - String json = EntityUtils.toString(response.getEntity(), "UTF-8"); - - JsonObject credentials = - JsonParser.parseString(json) - .getAsJsonObject() - .get("data") - .getAsJsonObject() - .getAsJsonObject("data"); - String username = credentials.get("username").getAsString(); - String password = credentials.get("password").getAsString(); - - Map config = new HashMap<>(); - config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_256.mechanismName()); - config.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.name()); - config.put( - SaslConfigs.SASL_JAAS_CONFIG, - String.format( - "org.apache.kafka.common.security.scram.ScramLoginModule required " - + "username=\"%s\" password=\"%s\";", - username, password)); - return config; + throws IOException { + // Execute a request to get the credentials + HttpClient client = HttpClientBuilder.create().build(); + HttpGet request = new HttpGet(secretStoreUrl); + request.addHeader("X-Vault-Token", token); + HttpResponse response = client.execute(request); + String json = EntityUtils.toString(response.getEntity(), "UTF-8"); - } catch (IOException exception) { - throw new KafkaException("Failed to retrieve credentials for Kafka client:", exception); - } + // Parse username and password from the response JSON + JsonObject credentials = + JsonParser.parseString(json) + .getAsJsonObject() + .get("data") + .getAsJsonObject() + .getAsJsonObject("data"); + String username = credentials.get("username").getAsString(); + String password = credentials.get("password").getAsString(); + + // Create the configuration for Kafka + Map config = new HashMap<>(); + config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_256.mechanismName()); + config.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.name()); + config.put( + SaslConfigs.SASL_JAAS_CONFIG, + String.format( + "org.apache.kafka.common.security.scram.ScramLoginModule required " + + "username=\"%s\" password=\"%s\";", + username, password)); + return config; } /** @@ -194,6 +207,7 @@ public static Map configureKafka(String secretStoreUrl, String t * @param options arguments to the pipeline */ public static PipelineResult run(KafkaToPubsubOptions options) { + // Configure Kafka consumer properties Map kafkaConfig = new HashMap<>(); try { String secretStoreUrl = options.getSecretStoreUrl(); @@ -202,7 +216,13 @@ public static PipelineResult run(KafkaToPubsubOptions options) { } catch (NullPointerException exception) { LOG.info( "No information to retrieve Kafka credentials was provided. " - + "Trying to initiate unauthorized connection."); + + "Trying to initiate an unauthorized connection."); + } catch (IOException exception) { + LOG.error( + String.format( + "Failed to retrieve credentials for Kafka client. " + + "Trying to initiate an unauthorized connection. Details: %s", + exception)); } List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); diff --git a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index 9823fc15da39..bd44627a7a72 100644 --- a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -22,6 +22,22 @@ "helpText": "Pub/Sub topic to write the output, in the format of 'projects/yourproject/topics/yourtopic'", "regexes": [], "paramType": "TEXT" + }, + { + "name": "secretStoreUrl", + "label": "URL to credentials in Vault", + "helpText": "URL to Kafka credentials in HashiCorp Vault secret storage in format 'http(s)://vaultip:vaultport/path/to/credentials'", + "regexes": [], + "paramType": "TEXT", + "isOptional": true + }, + { + "name": "vaultToken", + "label": "Vault token", + "helpText": "Token to access HashiCorp Vault secret storage", + "regexes": [], + "paramType": "TEXT", + "isOptional": true } ] } From e78c02cf851ee4fa634eee4b838b6752e6645d20 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Tue, 13 Oct 2020 17:09:26 +0300 Subject: [PATCH 22/73] Style fix --- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 22a6dc40de70..17a03a3641ee 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -58,9 +58,9 @@ public class KafkaToPubsub { * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format * from Kafka, and outputs the resulting records to PubSub. Input topics, output topic, Bootstrap * servers are specified by the user as template parameters.
    - * Kafka may be configured with SASL/SCRAM security mechanism, - * in this case a Vault secret storage with credentials should be provided. URL to credentials and Vault token - * are specified by the user as template parameters. + * Kafka may be configured with SASL/SCRAM security mechanism, in this case a Vault secret storage + * with credentials should be provided. URL to credentials and Vault token are specified by the + * user as template parameters. * *

    Pipeline Requirements * From 1055ec2c6e12f11fae67c6885721ba3f5e86d58f Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 14 Oct 2020 17:54:35 +0300 Subject: [PATCH 23/73] Added description for Vault parameters in metadata --- .../apache/beam/templates/options/KafkaToPubsubOptions.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 1690ec3b28d8..940ca6424643 100644 --- a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -43,12 +43,12 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setOutputTopic(String outputTopic); - @Description("") + @Description("URL to credentials in Vault") String getSecretStoreUrl(); void setSecretStoreUrl(String secretStoreUrl); - @Description("") + @Description("Vault token") String getVaultToken(); void setVaultToken(String vaultToken); From 77defc11c7707acd417b45a0779be634dcf8953f Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 15 Oct 2020 17:49:40 +0300 Subject: [PATCH 24/73] FIX trailing whitespaces in README.md --- templates/README.md | 28 ++++++++++++++-------------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/templates/README.md b/templates/README.md index a3aff1093503..1cc03acfe133 100644 --- a/templates/README.md +++ b/templates/README.md @@ -19,11 +19,11 @@ # Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline -to read data from a single or multiple topics from -[Apache Kafka](https://kafka.apache.org/) and write data into a single topic -in [Google Pub/Sub](https://cloud.google.com/pubsub). This template provides -availability to retrieve credentials for Apache Kafka from [HashiCorp Vault](https://www.vaultproject.io/) +This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline +to read data from a single or multiple topics from +[Apache Kafka](https://kafka.apache.org/) and write data into a single topic +in [Google Pub/Sub](https://cloud.google.com/pubsub). This template provides +availability to retrieve credentials for Apache Kafka from [HashiCorp Vault](https://www.vaultproject.io/) in case of configured SASL/SCRAM security mechanism. This template supports serializable string formats, such as JSON. @@ -48,7 +48,7 @@ This section describes what is needed to get the template up and running. ## Assembling the Uber-JAR -To run this template the template Java project should be built into +To run this template the template Java project should be built into an Uber JAR file. Navigate to the Beam folder: @@ -57,17 +57,17 @@ Navigate to the Beam folder: cd /path/to/beam ``` -In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) +In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) is used. It creates the `shadowJar` task that builds the Uber JAR: ``` ./gradlew -p templates/kafka-to-pubsub clean shadowJar ``` -ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains +ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains both target package *and* all its dependencies. -The result of the `shadowJar` task execution is a `.jar` file that is generated +The result of the `shadowJar` task execution is a `.jar` file that is generated under the `build/libs/` folder in kafka-to-pubsub directory. ## Local execution @@ -102,7 +102,7 @@ REGION=my-region #### Template Metadata Storage Bucket Creation -The Dataflow Flex template has to store its metadata in a bucket in +The Dataflow Flex template has to store its metadata in a bucket in [Google Cloud Storage](https://cloud.google.com/storage), so it can be executed from the Google Cloud Platform. Create the bucket in Google Cloud Storage if it doesn't exist yet: @@ -121,7 +121,7 @@ TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" ### Creating the Dataflow Flex Template -Dataflow Flex Templates package the pipeline as a Docker image and stage these images +Dataflow Flex Templates package the pipeline as a Docker image and stage these images on your project's [Container Registry](https://cloud.google.com/container-registry). To execute the template you need to create the template spec file containing all @@ -148,8 +148,8 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ ### Create Dataflow Job Using the Apache Kafka to Google Pub/Sub Dataflow Flex Template -To deploy the pipeline, you should refer to the template file and pass the -[parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) +To deploy the pipeline, you should refer to the template file and pass the +[parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) required by the pipeline. You can do this in 3 different ways: @@ -171,7 +171,7 @@ You can do this in 3 different ways: API_ROOT_URL="https://dataflow.googleapis.com" TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch" JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`" - + time curl -X POST -H "Content-Type: application/json" \ -H "Authorization: Bearer $(gcloud auth print-access-token)" \ "${TEMPLATES_LAUNCH_API}"` From d6ab0f6b13652260cb50b159575c253a7036b97f Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 15 Oct 2020 17:54:03 +0300 Subject: [PATCH 25/73] FIX. Blank line contains whitespace README.md --- templates/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/templates/README.md b/templates/README.md index 1cc03acfe133..a3ea135ace22 100644 --- a/templates/README.md +++ b/templates/README.md @@ -171,7 +171,7 @@ You can do this in 3 different ways: API_ROOT_URL="https://dataflow.googleapis.com" TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch" JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`" - + time curl -X POST -H "Content-Type: application/json" \ -H "Authorization: Bearer $(gcloud auth print-access-token)" \ "${TEMPLATES_LAUNCH_API}"` From 8881ff34a381cb3985aa0e2621c7a54fda8396e8 Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Thu, 15 Oct 2020 10:43:09 -0700 Subject: [PATCH 26/73] Update README.md --- templates/README.md | 11 +++++------ 1 file changed, 5 insertions(+), 6 deletions(-) diff --git a/templates/README.md b/templates/README.md index a3ea135ace22..7769a0037c08 100644 --- a/templates/README.md +++ b/templates/README.md @@ -22,11 +22,10 @@ This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic -in [Google Pub/Sub](https://cloud.google.com/pubsub). This template provides -availability to retrieve credentials for Apache Kafka from [HashiCorp Vault](https://www.vaultproject.io/) -in case of configured SASL/SCRAM security mechanism. +in [Google Pub/Sub](https://cloud.google.com/pubsub). -This template supports serializable string formats, such as JSON. +This template supports serializable string formats, such as JSON. The template supports Apache Kafka SASL/SCRAM security mechanisms +and retrieving credentials from [HashiCorp Vault](https://www.vaultproject.io/). ## Requirements @@ -34,7 +33,7 @@ This template supports serializable string formats, such as JSON. - Kafka Bootstrap Server(s) up and running - Existing source Kafka topic(s) - An existing Pub/Sub destination output topic -- (Optional) An existing HashiCorp Vault secret storage +- (Optional) An existing HashiCorp Vault ## Getting Started @@ -77,7 +76,7 @@ To execute this pipeline locally, specify the parameters: Kafka Bootstrap server --inputTopic=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame ``` -Optionally, specify a URL to credentials in Vault and token to access them: +Optionally, specify a URL to credentials in HashiCorp Vault and token to access them: ```bash --secretStoreUrl=http(s)://host:port/path/to/credentials --vaultToken=your-token From a821560f90e6a02928f30d132251dec9a22baa21 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Fri, 16 Oct 2020 15:15:37 +0300 Subject: [PATCH 27/73] Refactored to examples folder --- {templates => examples/templates/java}/README.md | 6 +++--- {templates => examples/templates/java}/build.gradle | 0 .../templates/java}/kafka-to-pubsub/build.gradle | 2 +- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 0 .../apache/beam/templates/options/KafkaToPubsubOptions.java | 0 .../org/apache/beam/templates/options/package-info.java | 0 .../main/java/org/apache/beam/templates/package-info.java | 0 .../src/main/resources/kafka_to_pubsub_metadata.json | 0 .../java/org/apache/beam/templates/KafkaToPubsubTest.java | 0 settings.gradle | 4 ++-- 10 files changed, 6 insertions(+), 6 deletions(-) rename {templates => examples/templates/java}/README.md (96%) rename {templates => examples/templates/java}/build.gradle (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/build.gradle (96%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json (100%) rename {templates => examples/templates/java}/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java (100%) diff --git a/templates/README.md b/examples/templates/java/README.md similarity index 96% rename from templates/README.md rename to examples/templates/java/README.md index 7769a0037c08..8ea994806ce8 100644 --- a/templates/README.md +++ b/examples/templates/java/README.md @@ -60,7 +60,7 @@ In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/john is used. It creates the `shadowJar` task that builds the Uber JAR: ``` -./gradlew -p templates/kafka-to-pubsub clean shadowJar +./gradlew -p examples/templates/java/kafka-to-pubsub clean shadowJar ``` ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains @@ -125,12 +125,12 @@ on your project's [Container Registry](https://cloud.google.com/container-regist To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has the following [metadata -file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. +file](examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: ``` -cd /path/to/beam/templates/kafka-to-pubsub +cd /path/to/beam/examples/templates/java/kafka-to-pubsub ``` Build the Dataflow Flex Template: diff --git a/templates/build.gradle b/examples/templates/java/build.gradle similarity index 100% rename from templates/build.gradle rename to examples/templates/java/build.gradle diff --git a/templates/kafka-to-pubsub/build.gradle b/examples/templates/java/kafka-to-pubsub/build.gradle similarity index 96% rename from templates/kafka-to-pubsub/build.gradle rename to examples/templates/java/kafka-to-pubsub/build.gradle index 34823e30aa0a..50eb7d433e01 100644 --- a/templates/kafka-to-pubsub/build.gradle +++ b/examples/templates/java/kafka-to-pubsub/build.gradle @@ -25,7 +25,7 @@ applyJavaNature( exportJavadoc: false, enableChecker: false, ignoreRawtypeErrors: true, - automaticModuleName: 'org.apache.beam.templates.kafka-to-pubsub') + automaticModuleName: 'org.apache.beam.examples.templates.kafka-to-pubsub') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java similarity index 100% rename from templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java similarity index 100% rename from templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java similarity index 100% rename from templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java diff --git a/templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java similarity index 100% rename from templates/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java diff --git a/templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json similarity index 100% rename from templates/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json rename to examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json diff --git a/templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java similarity index 100% rename from templates/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java rename to examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java diff --git a/settings.gradle b/settings.gradle index 22adb380a37d..998d1b969b40 100644 --- a/settings.gradle +++ b/settings.gradle @@ -214,6 +214,6 @@ include "beam-test-tools" project(":beam-test-tools").dir = file(".test-infra/tools") include "beam-test-jenkins" project(":beam-test-jenkins").dir = file(".test-infra/jenkins") -include ':templates' -include ':templates:kafka-to-pubsub' +include ':examples:templates:java' +include ':examples:templates:java:kafka-to-pubsub' From c22f110872aa19ba60e8b593556acabcd08381eb Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 16 Oct 2020 18:58:14 +0300 Subject: [PATCH 28/73] Added conversion from JSON into PubsubMessage and extracted all transformations from the pipeline class into the separate class --- examples/templates/java/README.md | 30 +-- .../apache/beam/templates/KafkaToPubsub.java | 185 ++++++++---------- .../options/KafkaToPubsubOptions.java | 9 + .../templates/transforms/FormatTransform.java | 95 +++++++++ .../templates/transforms/package-info.java | 20 ++ .../resources/kafka_to_pubsub_metadata.json | 8 + .../beam/templates/KafkaToPubsubTest.java | 3 +- 7 files changed, 231 insertions(+), 119 deletions(-) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 8ea994806ce8..fb8970844e1d 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -141,7 +141,7 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ --sdk-language "JAVA" \ --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ - --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \ + --jar "build/libs/beam-examples-templates-java-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \ --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" ``` @@ -168,27 +168,27 @@ You can do this in 3 different ways: 3. With a REST API request ``` API_ROOT_URL="https://dataflow.googleapis.com" - TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch" + TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/locations/${REGION}/flexTemplates:launch" JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`" - + time curl -X POST -H "Content-Type: application/json" \ -H "Authorization: Bearer $(gcloud auth print-access-token)" \ - "${TEMPLATES_LAUNCH_API}"` - `"?validateOnly=false"` - `"&dynamicTemplate.gcsPath=${BUCKET_NAME}/path/to/image-spec"` - `"&dynamicTemplate.stagingLocation=${BUCKET_NAME}/staging" \ -d ' { - "jobName":"${JOB_NAME}", - "parameters": { - "bootstrapServers":"broker_1:9092,broker_2:9092", - "inputTopics":"topic1,topic2", - "outputTopic":"projects/${PROJECT}/topics/your-topic-name", - "secretStoreUrl":"http(s)://host:port/path/to/credentials", - "vaultToken":"your-token" - } + "launch_parameter": { + "jobName": "'$JOB_NAME'", + "containerSpecGcsPath": "'$TEMPLATE_PATH'", + "parameters": { + "bootstrapServers": "broker_1:9091, broker_2:9092", + "inputTopics": "topic1, topic2", + "outputTopic": "projects/'$PROJECT'/topics/your-topic-name", + "secretStoreUrl": "http(s)://host:port/path/to/credentials", + "vaultToken": "your-token" + } + } } ' + "${TEMPLATES_LAUNCH_API}" ``` diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 17a03a3641ee..758cfbd991c8 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -29,17 +29,10 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; -import org.apache.beam.sdk.coders.NullableCoder; -import org.apache.beam.sdk.coders.StringUtf8Coder; -import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; -import org.apache.beam.sdk.io.kafka.KafkaIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; -import org.apache.beam.sdk.transforms.PTransform; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.sdk.values.KV; -import org.apache.beam.sdk.values.PBegin; -import org.apache.beam.sdk.values.PCollection; import org.apache.beam.templates.options.KafkaToPubsubOptions; +import org.apache.beam.templates.transforms.FormatTransform; import org.apache.http.HttpResponse; import org.apache.http.client.HttpClient; import org.apache.http.client.methods.HttpGet; @@ -48,90 +41,88 @@ import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.security.scram.ScramMechanism; -import org.apache.kafka.common.serialization.StringDeserializer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +/** + * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format from + * Kafka, and outputs the resulting records to PubSub. Input topics, output topic, Bootstrap servers + * are specified by the user as template parameters.
    + * Kafka may be configured with SASL/SCRAM security mechanism, in this case a Vault secret storage + * with credentials should be provided. URL to credentials and Vault token are specified by the user + * as template parameters. + * + *

    Pipeline Requirements + * + *

      + *
    • Kafka Bootstrap Server(s). + *
    • Kafka Topic(s) exists. + *
    • The PubSub output topic exists. + *
    • (Optional) An existing HashiCorp Vault secret storage + *
    + * + *

    Example Usage + * + *

    + * # Set the pipeline vars
    + * PROJECT=id-of-my-project
    + * BUCKET_NAME=my-bucket
    + *
    + * # Set containerization vars
    + * IMAGE_NAME=my-image-name
    + * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
    + * BASE_CONTAINER_IMAGE=my-base-container-image
    + * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json"
    + *
    + * # Create bucket in the cloud storage
    + * gsutil mb gs://${BUCKET_NAME}
    + *
    + * # Go to the beam folder
    + * cd /path/to/beam
    + *
    + * FLEX TEMPLATE
    + * # Assemble uber-jar
    + * ./gradlew -p templates/kafka-to-pubsub clean shadowJar
    + *
    + * # Go to the template folder
    + * cd /path/to/beam/templates/kafka-to-pubsub
    + *
    + * # Build the flex template
    + * gcloud dataflow flex-template build ${TEMPLATE_PATH} \
    + *       --image-gcr-path "${TARGET_GCR_IMAGE}" \
    + *       --sdk-language "JAVA" \
    + *       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
    + *       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
    + *       --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \
    + *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
    + *
    + * # Execute template:
    + *    API_ROOT_URL="https://dataflow.googleapis.com"
    + *    TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/locations/${REGION}/flexTemplates:launch"
    + *    JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`"
    + *
    + *    time curl -X POST -H "Content-Type: application/json" \
    + *            -H "Authorization: Bearer $(gcloud auth print-access-token)" \
    + *            -d '
    + *             {
    + *                 "launch_parameter": {
    + *                     "jobName": "'$JOB_NAME'",
    + *                     "containerSpecGcsPath": "'$TEMPLATE_PATH'",
    + *                     "parameters": {
    + *                         "bootstrapServers": "broker_1:9091, broker_2:9092",
    + *                         "inputTopics": "topic1, topic2",
    + *                         "outputTopic": "projects/'$PROJECT'/topics/your-topic-name",
    + *                         "secretStoreUrl": "http(s)://host:port/path/to/credentials",
    + *                         "vaultToken": "your-token"
    + *                     }
    + *                 }
    + *             }
    + *            '
    + *            "${TEMPLATES_LAUNCH_API}"
    + * 
    + */ public class KafkaToPubsub { - /** - * The {@link KafkaToPubsub} pipeline is a streaming pipeline which ingests data in JSON format - * from Kafka, and outputs the resulting records to PubSub. Input topics, output topic, Bootstrap - * servers are specified by the user as template parameters.
    - * Kafka may be configured with SASL/SCRAM security mechanism, in this case a Vault secret storage - * with credentials should be provided. URL to credentials and Vault token are specified by the - * user as template parameters. - * - *

    Pipeline Requirements - * - *

      - *
    • Kafka Bootstrap Server(s). - *
    • Kafka Topic(s) exists. - *
    • The PubSub output topic exists. - *
    • (Optional) An existing HashiCorp Vault secret storage - *
    - * - *

    Example Usage - * - *

    -   * # Set the pipeline vars
    -   * PROJECT=id-of-my-project
    -   * BUCKET_NAME=my-bucket
    -   *
    -   * # Set containerization vars
    -   * IMAGE_NAME=my-image-name
    -   * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
    -   * BASE_CONTAINER_IMAGE=my-base-container-image
    -   * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json"
    -   *
    -   * # Create bucket in the cloud storage
    -   * gsutil mb gs://${BUCKET_NAME}
    -   *
    -   * # Go to the beam folder
    -   * cd /path/to/beam
    -   *
    -   * FLEX TEMPLATE
    -   * # Assemble uber-jar
    -   * ./gradlew -p templates/kafka-to-pubsub clean shadowJar
    -   *
    -   * # Go to the template folder
    -   * cd /path/to/beam/templates/kafka-to-pubsub
    -   *
    -   * # Build the flex template
    -   * gcloud dataflow flex-template build ${TEMPLATE_PATH} \
    -   *       --image-gcr-path "${TARGET_GCR_IMAGE}" \
    -   *       --sdk-language "JAVA" \
    -   *       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
    -   *       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
    -   *       --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \
    -   *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
    -   *
    -   * # Execute template:
    -   * API_ROOT_URL="https://dataflow.googleapis.com"
    -   * TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/templates:launch"
    -   * JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`"
    -   *
    -   * time curl -X POST -H "Content-Type: application/json"     \
    -   *     -H "Authorization: Bearer $(gcloud auth print-access-token)" \
    -   *     "${TEMPLATES_LAUNCH_API}"`
    -   *     `"?validateOnly=false"`
    -   *     `"&dynamicTemplate.gcsPath=${BUCKET_NAME}/path/to/image-spec"`
    -   *     `"&dynamicTemplate.stagingLocation=${BUCKET_NAME}/staging" \
    -   *     -d '
    -   *      {
    -   *       "jobName":"${JOB_NAME}",
    -   *       "parameters": {
    -   *           "bootstrapServers":"broker_1:9092,broker_2:9092",
    -   *           "inputTopics":"topic1,topic2",
    -   *           "outputTopic":"projects/${PROJECT}/topics/your-topic-name",
    -   *           "secretStoreUrl":"http(s)://host:port/path/to/credentials",
    -   *           "vaultToken":"your-token"
    -   *        }
    -   *       }
    -   *      '
    -   * 
    - */ - /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(KafkaToPubsub.class); @@ -147,19 +138,6 @@ public static void main(String[] args) { run(options); } - public static PTransform>> readFromKafka( - String bootstrapServers, List topicsList, Map config) { - return KafkaIO.read() - .withBootstrapServers(bootstrapServers) - .withTopics(topicsList) - .withKeyDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withValueDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withConsumerConfigUpdates(config) - .withoutMetadata(); - } - /** * Retrieves username and password from HashiCorp Vault secret storage and configures Kafka * consumer for authorized connection. @@ -256,9 +234,10 @@ public static PipelineResult run(KafkaToPubsubOptions options) { */ pipeline .apply( - "ReadFromKafka", readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) - .apply(Values.create()) - .apply("Write PubSub Events", PubsubIO.writeStrings().to(options.getOutputTopic())); + "readFromKafka", + FormatTransform.readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) + .apply("createValues", Values.create()) + .apply("writeToPubSub", new FormatTransform.FormatOutput(options)); return pipeline.run(); } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 940ca6424643..c332ea02786b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -17,9 +17,11 @@ */ package org.apache.beam.templates.options; +import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; +import org.apache.beam.templates.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { @Description("Kafka Bootstrap Servers") @@ -43,6 +45,13 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setOutputTopic(String outputTopic); + @Description("") + @Validation.Required + @Default.Enum("JSON") + FormatTransform.FORMAT getOutputFormat(); + + void setOutputFormat(FormatTransform.FORMAT outputFormat); + @Description("URL to credentials in Vault") String getSecretStoreUrl(); diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java new file mode 100644 index 000000000000..0fabe3f0d4f1 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -0,0 +1,95 @@ +/* + * 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.templates.transforms; + +import java.util.List; +import java.util.Map; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubMessage; +import org.apache.beam.sdk.io.kafka.KafkaIO; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.transforms.PTransform; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.PBegin; +import org.apache.beam.sdk.values.PCollection; +import org.apache.beam.sdk.values.PDone; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.templates.options.KafkaToPubsubOptions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; +import org.apache.kafka.common.serialization.StringDeserializer; + +/** Different transformations over the processed data in the pipeline. */ +public class FormatTransform { + + public enum FORMAT { + JSON, + AVRO + } + + /** + * Configures Kafka consumer. + * + * @param bootstrapServers Kafka servers to read from + * @param topicsList Kafka topics to read from + * @param config configuration for the Kafka consumer + * @return configured reading from Kafka + */ + public static PTransform>> readFromKafka( + String bootstrapServers, List topicsList, Map config) { + return KafkaIO.read() + .withBootstrapServers(bootstrapServers) + .withTopics(topicsList) + .withKeyDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withValueDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withConsumerConfigUpdates(config) + .withoutMetadata(); + } + + /** Converts all strings into a chosen {@link FORMAT} and writes them into PubSub topic. */ + public static class FormatOutput extends PTransform, PDone> { + + private final KafkaToPubsubOptions options; + + public FormatOutput(KafkaToPubsubOptions options) { + this.options = options; + } + + @Override + public PDone expand(PCollection input) { + if (options.getOutputFormat() == FORMAT.JSON) { + return input + .apply( + "convertMessagesToPubsubMessages", + MapElements.into(TypeDescriptor.of(PubsubMessage.class)) + .via( + (String json) -> + new PubsubMessage(json.getBytes(Charsets.UTF_8), ImmutableMap.of()))) + .apply( + "writePubsubMessagesToPubSub", + PubsubIO.writeMessages().to(options.getOutputTopic())); + } else { + return input.apply("writeToPubSub", PubsubIO.writeStrings().to(options.getOutputTopic())); + } + } + } +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java new file mode 100644 index 000000000000..7d97ed60f7af --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template for Google Cloud Dataflow. */ +package org.apache.beam.templates.transforms; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index bd44627a7a72..a1e226fc7fc6 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -23,6 +23,14 @@ "regexes": [], "paramType": "TEXT" }, + { + "name": "outputFormat", + "label": "Message format that is written into Pub/Sub", + "helpText": "One of two message formats that is written to Pub/Sub. JSON or AVRO", + "regexes": [ + "^(JSON|AVRO)$" + ] + }, { "name": "secretStoreUrl", "label": "URL to credentials in Vault", diff --git a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java index 89a1e36aa2b0..489e5e4591d8 100644 --- a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java +++ b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java @@ -23,6 +23,7 @@ import java.util.List; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.testing.TestPipeline; +import org.apache.beam.templates.transforms.FormatTransform; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; @@ -43,7 +44,7 @@ public void testKafkaReadingFailsWrongBootstrapServer() { final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); final HashMap config = new HashMap<>(); - pipeline.apply(KafkaToPubsub.readFromKafka(bootstrapServers, topicsList, config)); + pipeline.apply(FormatTransform.readFromKafka(bootstrapServers, topicsList, config)); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage("Failed to construct kafka consumer"); pipeline.run(); From b56ec7bd557ad965b7ce8c0fdcaa5331a8858245 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 16 Oct 2020 19:37:37 +0300 Subject: [PATCH 29/73] Whitespacelint fix --- examples/templates/java/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index fb8970844e1d..36ed35152399 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -125,7 +125,7 @@ on your project's [Container Registry](https://cloud.google.com/container-regist To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has the following [metadata -file](examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. +file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: @@ -170,7 +170,7 @@ You can do this in 3 different ways: API_ROOT_URL="https://dataflow.googleapis.com" TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/locations/${REGION}/flexTemplates:launch" JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`" - + time curl -X POST -H "Content-Type: application/json" \ -H "Authorization: Bearer $(gcloud auth print-access-token)" \ -d ' From 71308e573cd3f9b94913534020ddd76d67eba748 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 21 Oct 2020 18:09:13 +0300 Subject: [PATCH 30/73] Updated README.md and output formats --- examples/templates/java/README.md | 77 ++++++++++++++++--- .../templates/transforms/FormatTransform.java | 6 +- .../resources/kafka_to_pubsub_metadata.json | 7 +- 3 files changed, 75 insertions(+), 15 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 36ed35152399..ee8b5b8cfec3 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -24,7 +24,8 @@ to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). -This template supports serializable string formats, such as JSON. The template supports Apache Kafka SASL/SCRAM security mechanisms +This template supports serializable plaintext formats and [PubSubMessage](https://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage). +The template supports Apache Kafka SASL/SCRAM security mechanisms via PLAINTEXT or SSL and retrieving credentials from [HashiCorp Vault](https://www.vaultproject.io/). ## Requirements @@ -34,6 +35,7 @@ and retrieving credentials from [HashiCorp Vault](https://www.vaultproject.io/). - Existing source Kafka topic(s) - An existing Pub/Sub destination output topic - (Optional) An existing HashiCorp Vault +- (Optional) A configured secure SSL connection for Kafka ## Getting Started @@ -70,25 +72,51 @@ The result of the `shadowJar` task execution is a `.jar` file that is generated under the `build/libs/` folder in kafka-to-pubsub directory. ## Local execution -To execute this pipeline locally, specify the parameters: Kafka Bootstrap servers, Kafka input topic, Pub/Sub output topic with the form: +To execute this pipeline locally, specify the parameters: +- Kafka Bootstrap servers +- Kafka input topics +- Pub/Sub output topic with the form ```bash --bootstrapServers=host:port \ ---inputTopic=your-input-topic \ +--inputTopics=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame ``` -Optionally, specify a URL to credentials in HashiCorp Vault and token to access them: +Optionally, to choose the output format, specify one of two available formats: +- PLAINTEXT (default) +- PUBSUB +```bash +--outputFormat=PLAINTEXT +``` +Optionally, to retrieve Kafka credentials for SASL/SCRAM, +specify a URL to credentials in HashiCorp Vault and token to access them: ```bash --secretStoreUrl=http(s)://host:port/path/to/credentials --vaultToken=your-token ``` +Optionally, to configure secure SSL connection between the pipeline and Kafka, +specify the parameters: +- A local path to a truststore file +- A local path to a keystore file +- Truststore password +- Keystore password +- Key password +```bash +--truststorePath=path/to/kafka.truststore.jks +--keystorePath=path/to/kafka.keystore.jks +--truststorePassword=your-truststore-password +--keystorePassword=your-keystore-password +--keyPassword=your-key-password +``` To change the runner, specify: - -```--runner=YOUR_SELECTED_RUNNER``` - +```bash +--runner=YOUR_SELECTED_RUNNER +``` See examples/java/README.md for instructions about how to configure different runners. ## Google Dataflow Template +_Note: The Dataflow Template doesn't support SSL configuration._ + ### Setting Up Project Environment #### Pipeline variables: @@ -161,8 +189,9 @@ You can do this in 3 different ways: --parameters bootstrapServers="broker_1:9092,broker_2:9092" \ --parameters inputTopics="topic1,topic2" \ --parameters outputTopic="projects/${PROJECT}/topics/your-topic-name" \ - --parameters secretStoreUrl="http(s)://host:port/path/to/credentials" - --parameters vaultToken="your-token" + --parameters outputFormat="PLAINTEXT" \ + --parameters secretStoreUrl="http(s)://host:port/path/to/credentials" \ + --parameters vaultToken="your-token" \ --region "${REGION}" ``` 3. With a REST API request @@ -182,6 +211,7 @@ You can do this in 3 different ways: "bootstrapServers": "broker_1:9091, broker_2:9092", "inputTopics": "topic1, topic2", "outputTopic": "projects/'$PROJECT'/topics/your-topic-name", + "outputFormat": "PLAINTEXT", "secretStoreUrl": "http(s)://host:port/path/to/credentials", "vaultToken": "your-token" } @@ -191,4 +221,33 @@ You can do this in 3 different ways: "${TEMPLATES_LAUNCH_API}" ``` +## Use Cases + +Generally, the whole range of use cases can be described via the simplest and +the most complex examples. All other use cases can be described as different +variations between the simplest and the most complex ones. + +### The simplest use case + +There are: +- One Kafka server without SASL and SSL +- One topic in Kafka +- One topic in Google Pub/Sub + +Kafka can be reached without any authentication and SSL certificates. +The pipeline streams plaintext messages from Kafka topic to Google Pub/Sub topic. + +### The most complex use case + +There are: +- Several Kafka servers with configured SASL/SCRAM and SSL secure connection +- Several topics in Kafka with same username and password +- One topic in Google Pub/Sub +- Unsealed HashiCorp Vault secret storage with known secret access token +- SSL certificate located locally +The pipeline reaches Vault secret storage via access token and +retrieves the credentials that contain username and password for Kafka topics. +Using the SSL certificate and retrieved credentials the pipeline connects to +Kafka and streams messages from Kafka topics into Google Pub/Sub converting them +into PubSubMessages. diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index 0fabe3f0d4f1..e61e3568f644 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -40,8 +40,8 @@ public class FormatTransform { public enum FORMAT { - JSON, - AVRO + PUBSUB, + PLAINTEXT } /** @@ -76,7 +76,7 @@ public FormatOutput(KafkaToPubsubOptions options) { @Override public PDone expand(PCollection input) { - if (options.getOutputFormat() == FORMAT.JSON) { + if (options.getOutputFormat() == FORMAT.PUBSUB) { return input .apply( "convertMessagesToPubsubMessages", diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index a1e226fc7fc6..1830234b4955 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -26,10 +26,11 @@ { "name": "outputFormat", "label": "Message format that is written into Pub/Sub", - "helpText": "One of two message formats that is written to Pub/Sub. JSON or AVRO", + "helpText": "One of two message formats that is written to Pub/Sub. PUBSUB or PLAINTEXT. Default is PLAINTEXT", "regexes": [ - "^(JSON|AVRO)$" - ] + "^(PUBSUB|PLAINTEXT)$" + ], + "isOptional": true }, { "name": "secretStoreUrl", From 3c218dfc039276c684fe2218bf3a326b76bcd791 Mon Sep 17 00:00:00 2001 From: AKosolapov Date: Thu, 22 Oct 2020 00:11:43 -0700 Subject: [PATCH 31/73] Update README.md --- examples/templates/java/README.md | 59 ++++++++++--------------------- 1 file changed, 19 insertions(+), 40 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index ee8b5b8cfec3..967305e2db2a 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -24,9 +24,19 @@ to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Pub/Sub](https://cloud.google.com/pubsub). -This template supports serializable plaintext formats and [PubSubMessage](https://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage). -The template supports Apache Kafka SASL/SCRAM security mechanisms via PLAINTEXT or SSL -and retrieving credentials from [HashiCorp Vault](https://www.vaultproject.io/). +Supported data formats: +- Serializable plaintext formats, such as JSON +- [PubSubMessage](https://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage). + +Supported input source configurations: +- Single or multiple Apache Kafka bootstrap servers +- Apache Kafka SASL/SCRAM authentication over plaintext or SSL connection +- Secrets vault service [HashiCorp Vault](https://www.vaultproject.io/). + +Supported destination configuration: +- Single Google Pub/Sub topic. + +In a simple scenario, the template will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The template supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the template will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. ## Requirements @@ -75,7 +85,8 @@ under the `build/libs/` folder in kafka-to-pubsub directory. To execute this pipeline locally, specify the parameters: - Kafka Bootstrap servers - Kafka input topics -- Pub/Sub output topic with the form +- Pub/Sub output topic +in the followin format: ```bash --bootstrapServers=host:port \ --inputTopics=your-input-topic \ @@ -88,12 +99,12 @@ Optionally, to choose the output format, specify one of two available formats: --outputFormat=PLAINTEXT ``` Optionally, to retrieve Kafka credentials for SASL/SCRAM, -specify a URL to credentials in HashiCorp Vault and token to access them: +specify a URL to the credentials in HashiCorp Vault and the vault access token: ```bash --secretStoreUrl=http(s)://host:port/path/to/credentials --vaultToken=your-token ``` -Optionally, to configure secure SSL connection between the pipeline and Kafka, +Optionally, to configure secure SSL connection between the Beam pipeline and Kafka, specify the parameters: - A local path to a truststore file - A local path to a keystore file @@ -111,12 +122,10 @@ To change the runner, specify: ```bash --runner=YOUR_SELECTED_RUNNER ``` -See examples/java/README.md for instructions about how to configure different runners. +See examples/java/README.md for steps and examples to configure different runners. ## Google Dataflow Template -_Note: The Dataflow Template doesn't support SSL configuration._ - ### Setting Up Project Environment #### Pipeline variables: @@ -220,34 +229,4 @@ You can do this in 3 different ways: ' "${TEMPLATES_LAUNCH_API}" ``` - -## Use Cases - -Generally, the whole range of use cases can be described via the simplest and -the most complex examples. All other use cases can be described as different -variations between the simplest and the most complex ones. - -### The simplest use case - -There are: -- One Kafka server without SASL and SSL -- One topic in Kafka -- One topic in Google Pub/Sub - -Kafka can be reached without any authentication and SSL certificates. -The pipeline streams plaintext messages from Kafka topic to Google Pub/Sub topic. - -### The most complex use case - -There are: -- Several Kafka servers with configured SASL/SCRAM and SSL secure connection -- Several topics in Kafka with same username and password -- One topic in Google Pub/Sub -- Unsealed HashiCorp Vault secret storage with known secret access token -- SSL certificate located locally - -The pipeline reaches Vault secret storage via access token and -retrieves the credentials that contain username and password for Kafka topics. -Using the SSL certificate and retrieved credentials the pipeline connects to -Kafka and streams messages from Kafka topics into Google Pub/Sub converting them -into PubSubMessages. + _Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ From 8bdfc3413ae39c7f3451a675b15ae41a56d4689a Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Thu, 22 Oct 2020 17:49:44 +0300 Subject: [PATCH 32/73] Update README.md --- examples/templates/java/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 967305e2db2a..737c116f5179 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -86,7 +86,7 @@ To execute this pipeline locally, specify the parameters: - Kafka Bootstrap servers - Kafka input topics - Pub/Sub output topic -in the followin format: +in the following format: ```bash --bootstrapServers=host:port \ --inputTopics=your-input-topic \ From 1d94fcdd79cd485b1d8193c443f72dcd88da6df6 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Thu, 22 Oct 2020 19:13:13 +0300 Subject: [PATCH 33/73] Added support for SSL and removed outputFormat option --- examples/templates/java/README.md | 13 +-- .../beam/templates/ConsumerFactoryFn.java | 82 +++++++++++++++++++ .../apache/beam/templates/KafkaToPubsub.java | 33 ++++++-- .../options/KafkaToPubsubOptions.java | 34 ++++++-- .../templates/transforms/FormatTransform.java | 45 +++++----- .../beam/templates/KafkaToPubsubTest.java | 6 +- 6 files changed, 164 insertions(+), 49 deletions(-) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 737c116f5179..912d16a9af68 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -92,12 +92,6 @@ in the following format: --inputTopics=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame ``` -Optionally, to choose the output format, specify one of two available formats: -- PLAINTEXT (default) -- PUBSUB -```bash ---outputFormat=PLAINTEXT -``` Optionally, to retrieve Kafka credentials for SASL/SCRAM, specify a URL to the credentials in HashiCorp Vault and the vault access token: ```bash @@ -161,8 +155,8 @@ Dataflow Flex Templates package the pipeline as a Docker image and stage these i on your project's [Container Registry](https://cloud.google.com/container-registry). To execute the template you need to create the template spec file containing all -the necessary information to run the job. This template already has the following [metadata -file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. +the necessary information to run the job. This template already has the following +[metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: @@ -229,4 +223,5 @@ You can do this in 3 different ways: ' "${TEMPLATES_LAUNCH_API}" ``` - _Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ + +_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java new file mode 100644 index 000000000000..ff3ecee353f7 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java @@ -0,0 +1,82 @@ +/* + * 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.templates; + +import java.io.File; +import java.io.IOException; +import java.util.Map; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.security.auth.SecurityProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Class to create Kafka Consumer with configured SSL. */ +public class ConsumerFactoryFn + implements SerializableFunction, Consumer> { + private final Map sslConfig; + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(ConsumerFactoryFn.class); + + public ConsumerFactoryFn(Map sslConfig) { + this.sslConfig = sslConfig; + } + + @Override + public Consumer apply(Map config) { + try { + checkFileExists(sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + checkFileExists(sslConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)); + } catch (IOException | NullPointerException e) { + LOG.error("Failed to retrieve data for SSL", e); + return new KafkaConsumer<>(config); + } + + config.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SecurityProtocol.SASL_SSL.name()); + config.put( + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, + sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + config.put( + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, + sslConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)); + config.put( + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, + sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + config.put( + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, + sslConfig.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG)); + config.put( + SslConfigs.SSL_KEY_PASSWORD_CONFIG, sslConfig.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG)); + + return new KafkaConsumer<>(config); + } + + private void checkFileExists(String filePath) throws IOException { + File f = new File(filePath); + if (f.exists()) { + LOG.debug("{} exists", f.getAbsolutePath()); + } else { + LOG.error("{} does not exist", f.getAbsolutePath()); + throw new IOException(); + } + } +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 758cfbd991c8..3b3aa2e63a64 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -39,6 +39,7 @@ import org.apache.http.impl.client.HttpClientBuilder; import org.apache.http.util.EntityUtils; import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.security.auth.SecurityProtocol; import org.apache.kafka.common.security.scram.ScramMechanism; import org.slf4j.Logger; @@ -168,7 +169,7 @@ public static Map configureKafka(String secretStoreUrl, String t // Create the configuration for Kafka Map config = new HashMap<>(); - config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_256.mechanismName()); + config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_512.mechanismName()); config.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.name()); config.put( SaslConfigs.SASL_JAAS_CONFIG, @@ -179,6 +180,17 @@ public static Map configureKafka(String secretStoreUrl, String t return config; } + public static Map configureSsl(KafkaToPubsubOptions options) { + Map config = new HashMap<>(); + config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, options.getTruststorePath()); + config.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, options.getKeystorePath()); + config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, options.getTruststorePassword()); + config.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, options.getKeystorePassword()); + config.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, options.getKeyPassword()); + + return config; + } + /** * Runs a pipeline which reads message from Kafka and writes it to GCS. * @@ -197,10 +209,18 @@ public static PipelineResult run(KafkaToPubsubOptions options) { + "Trying to initiate an unauthorized connection."); } catch (IOException exception) { LOG.error( - String.format( - "Failed to retrieve credentials for Kafka client. " - + "Trying to initiate an unauthorized connection. Details: %s", - exception)); + "Failed to retrieve credentials for Kafka client. " + + "Trying to initiate an unauthorized connection.", + exception); + } + + Map sslConfig = new HashMap<>(); + try { + sslConfig.putAll(configureSsl(options)); + } catch (NullPointerException e) { + LOG.info( + "No information to retrieve SSL certificate was provided. " + + "Trying to initiate a plain text connection."); } List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); @@ -235,7 +255,8 @@ public static PipelineResult run(KafkaToPubsubOptions options) { pipeline .apply( "readFromKafka", - FormatTransform.readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) + FormatTransform.readFromKafka( + options.getBootstrapServers(), topicsList, kafkaConfig, sslConfig)) .apply("createValues", Values.create()) .apply("writeToPubSub", new FormatTransform.FormatOutput(options)); diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index c332ea02786b..339350cc4b09 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -17,11 +17,9 @@ */ package org.apache.beam.templates.options; -import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; -import org.apache.beam.templates.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { @Description("Kafka Bootstrap Servers") @@ -45,13 +43,6 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setOutputTopic(String outputTopic); - @Description("") - @Validation.Required - @Default.Enum("JSON") - FormatTransform.FORMAT getOutputFormat(); - - void setOutputFormat(FormatTransform.FORMAT outputFormat); - @Description("URL to credentials in Vault") String getSecretStoreUrl(); @@ -61,4 +52,29 @@ public interface KafkaToPubsubOptions extends PipelineOptions { String getVaultToken(); void setVaultToken(String vaultToken); + + @Description("The path to the trust store file") + String getTruststorePath(); + + void setTruststorePath(String truststorePath); + + @Description("The path to the key store file") + String getKeystorePath(); + + void setKeystorePath(String keystorePath); + + @Description("The password for the trust store file") + String getTruststorePassword(); + + void setTruststorePassword(String truststorePassword); + + @Description("The store password for the key store file") + String getKeystorePassword(); + + void setKeystorePassword(String keystorePassword); + + @Description("The password of the private key in the key store file") + String getKeyPassword(); + + void setKeyPassword(String keyPassword); } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index e61e3568f644..2a0a31819bf6 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -31,6 +31,7 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.templates.ConsumerFactoryFn; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -39,21 +40,20 @@ /** Different transformations over the processed data in the pipeline. */ public class FormatTransform { - public enum FORMAT { - PUBSUB, - PLAINTEXT - } - /** * Configures Kafka consumer. * * @param bootstrapServers Kafka servers to read from * @param topicsList Kafka topics to read from - * @param config configuration for the Kafka consumer + * @param kafkaConfig configuration for the Kafka consumer + * @param sslConfig configuration for the SSL connection * @return configured reading from Kafka */ public static PTransform>> readFromKafka( - String bootstrapServers, List topicsList, Map config) { + String bootstrapServers, + List topicsList, + Map kafkaConfig, + Map sslConfig) { return KafkaIO.read() .withBootstrapServers(bootstrapServers) .withTopics(topicsList) @@ -61,11 +61,15 @@ public static PTransform>> readFromKafka( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) .withValueDeserializerAndCoder( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) - .withConsumerConfigUpdates(config) + .withConsumerConfigUpdates(kafkaConfig) + .withConsumerFactoryFn(new ConsumerFactoryFn(sslConfig)) .withoutMetadata(); } - /** Converts all strings into a chosen {@link FORMAT} and writes them into PubSub topic. */ + /** + * The {@link FormatOutput} wraps a String serializable messages with the {@link PubsubMessage} + * class. + */ public static class FormatOutput extends PTransform, PDone> { private final KafkaToPubsubOptions options; @@ -76,20 +80,15 @@ public FormatOutput(KafkaToPubsubOptions options) { @Override public PDone expand(PCollection input) { - if (options.getOutputFormat() == FORMAT.PUBSUB) { - return input - .apply( - "convertMessagesToPubsubMessages", - MapElements.into(TypeDescriptor.of(PubsubMessage.class)) - .via( - (String json) -> - new PubsubMessage(json.getBytes(Charsets.UTF_8), ImmutableMap.of()))) - .apply( - "writePubsubMessagesToPubSub", - PubsubIO.writeMessages().to(options.getOutputTopic())); - } else { - return input.apply("writeToPubSub", PubsubIO.writeStrings().to(options.getOutputTopic())); - } + return input + .apply( + "convertMessagesToPubsubMessages", + MapElements.into(TypeDescriptor.of(PubsubMessage.class)) + .via( + (String json) -> + new PubsubMessage(json.getBytes(Charsets.UTF_8), ImmutableMap.of()))) + .apply( + "writePubsubMessagesToPubSub", PubsubIO.writeMessages().to(options.getOutputTopic())); } } } diff --git a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java index 489e5e4591d8..35e496b39850 100644 --- a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java +++ b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java @@ -42,9 +42,11 @@ public class KafkaToPubsubTest { public void testKafkaReadingFailsWrongBootstrapServer() { final String bootstrapServers = "some-server:9092"; final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); - final HashMap config = new HashMap<>(); + final HashMap kafkaConfig = new HashMap<>(); + final HashMap sslConfig = new HashMap<>(); - pipeline.apply(FormatTransform.readFromKafka(bootstrapServers, topicsList, config)); + pipeline.apply( + FormatTransform.readFromKafka(bootstrapServers, topicsList, kafkaConfig, sslConfig)); thrown.expect(Pipeline.PipelineExecutionException.class); thrown.expectMessage("Failed to construct kafka consumer"); pipeline.run(); From a194d54ffc9a54d7a76607203bbb1a3a5dd963f4 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 22 Oct 2020 23:31:05 +0300 Subject: [PATCH 34/73] Added avro usage example --- .../apache/beam/templates/KafkaToPubsub.java | 26 +++- .../apache/beam/templates/avro/TaxiRide.java | 130 ++++++++++++++++++ .../avro/TaxiRidesKafkaAvroDeserializer.java | 41 ++++++ .../beam/templates/avro/package-info.java | 20 +++ .../beam/templates/options/package-info.java | 2 +- .../apache/beam/templates/package-info.java | 2 +- .../templates/transforms/FormatTransform.java | 27 +++- .../templates/transforms/package-info.java | 2 +- 8 files changed, 240 insertions(+), 10 deletions(-) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 758cfbd991c8..a5285d5e3e00 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -29,8 +29,10 @@ import java.util.Map; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.templates.avro.TaxiRide; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.templates.transforms.FormatTransform; import org.apache.http.HttpResponse; @@ -232,12 +234,24 @@ public static PipelineResult run(KafkaToPubsubOptions options) { * 2) Extract values only * 3) Write successful records to PubSub */ - pipeline - .apply( - "readFromKafka", - FormatTransform.readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) - .apply("createValues", Values.create()) - .apply("writeToPubSub", new FormatTransform.FormatOutput(options)); + + if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { + pipeline + .apply( + "readAvrosFromKafka", + FormatTransform.readAvrosFromKafka( + options.getBootstrapServers(), topicsList, kafkaConfig)) + .apply("createValues", Values.create()) + .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); + + } else { + pipeline + .apply( + "readFromKafka", + FormatTransform.readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig)) + .apply("createValues", Values.create()) + .apply("writeToPubSub", new FormatTransform.FormatOutput(options)); + } return pipeline.run(); } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java new file mode 100644 index 000000000000..c669cffcdd33 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java @@ -0,0 +1,130 @@ +/* + * 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.templates.avro; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; + +/** + * Example of AVRO serialization class. To configure your AVRO schema, change this class to + * requirement schema definition + */ +@DefaultCoder(AvroCoder.class) +public class TaxiRide { + // "ride_id":"a60ba4d8-1501-4b5b-93ee-b7864304d0e0", + // "latitude":40.66684000000033, + // "longitude":-73.83933000000202, + // "timestamp":"2016-08-31T11:04:02.025396463-04:00", + // "meter_reading":14.270274, + // "meter_increment":0.019336415, + // "ride_status":"enroute" / "pickup" / "dropoff" + // "passenger_count":2 + String rideId; + Float latitude; + Float longitude; + String timestamp; + Float meterReading; + Float meterIncrement; + String rideStatus; + Integer passengerCount; + + public TaxiRide() {} + + public TaxiRide( + String rideId, + Float latitude, + Float longitude, + String timestamp, + Float meterReading, + Float meterIncrement, + String rideStatus, + Integer passengerCount) { + this.rideId = rideId; + this.latitude = latitude; + this.longitude = longitude; + this.timestamp = timestamp; + this.meterReading = meterReading; + this.meterIncrement = meterIncrement; + this.rideStatus = rideStatus; + this.passengerCount = passengerCount; + } + + public String getRideId() { + return rideId; + } + + public void setRideId(String rideId) { + this.rideId = rideId; + } + + public Float getLatitude() { + return latitude; + } + + public void setLatitude(Float latitude) { + this.latitude = latitude; + } + + public Float getLongitude() { + return longitude; + } + + public void setLongitude(Float longitude) { + this.longitude = longitude; + } + + public String getTimestamp() { + return timestamp; + } + + public void setTimestamp(String timestamp) { + this.timestamp = timestamp; + } + + public Float getMeterReading() { + return meterReading; + } + + public void setMeterReading(Float meterReading) { + this.meterReading = meterReading; + } + + public Float getMeterIncrement() { + return meterIncrement; + } + + public void setMeterIncrement(Float meterIncrement) { + this.meterIncrement = meterIncrement; + } + + public String getRideStatus() { + return rideStatus; + } + + public void setRideStatus(String rideStatus) { + this.rideStatus = rideStatus; + } + + public Integer getPassengerCount() { + return passengerCount; + } + + public void setPassengerCount(Integer passengerCount) { + this.passengerCount = passengerCount; + } +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java new file mode 100644 index 000000000000..474c032bb1d4 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java @@ -0,0 +1,41 @@ +/* + * 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.templates.avro; + +import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; +import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; +import java.util.Map; +import org.apache.kafka.common.serialization.Deserializer; + +/** Example of custom AVRO Deserialize. */ +public class TaxiRidesKafkaAvroDeserializer extends AbstractKafkaAvroDeserializer + implements Deserializer { + + @Override + public void configure(Map configs, boolean isKey) { + configure(new KafkaAvroDeserializerConfig(configs)); + } + + @Override + public TaxiRide deserialize(String s, byte[] bytes) { + return (TaxiRide) this.deserialize(bytes); + } + + @Override + public void close() {} +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java new file mode 100644 index 000000000000..0a3040b226eb --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template. */ +package org.apache.beam.templates.avro; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java index 5c9fa57f8ed2..eb553043ce1f 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java @@ -16,5 +16,5 @@ * limitations under the License. */ -/** Kafka to Pubsub template for Google Cloud Dataflow. */ +/** Kafka to Pubsub template. */ package org.apache.beam.templates.options; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java index 4e7b8ca2b97b..0a83c30478ce 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java @@ -16,5 +16,5 @@ * limitations under the License. */ -/** Kafka to Pubsub template for Google Cloud Dataflow. */ +/** Kafka to Pubsub template. */ package org.apache.beam.templates; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index e61e3568f644..bb25f76156ee 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -19,6 +19,7 @@ import java.util.List; import java.util.Map; +import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; @@ -31,6 +32,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.templates.avro.TaxiRide; +import org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -41,7 +44,8 @@ public class FormatTransform { public enum FORMAT { PUBSUB, - PLAINTEXT + PLAINTEXT, + AVRO; } /** @@ -65,6 +69,27 @@ public static PTransform>> readFromKafka( .withoutMetadata(); } + /** + * Configures Kafka consumer to read avros to {@link TaxiRide} format. + * + * @param bootstrapServers Kafka servers to read from + * @param topicsList Kafka topics to read from + * @param config configuration for the Kafka consumer + * @return configured reading from Kafka + */ + public static PTransform>> readAvrosFromKafka( + String bootstrapServers, List topicsList, Map config) { + return KafkaIO.read() + .withBootstrapServers(bootstrapServers) + .withTopics(topicsList) + .withKeyDeserializerAndCoder( + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + .withValueDeserializerAndCoder( + TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) + .withConsumerConfigUpdates(config) + .withoutMetadata(); + } + /** Converts all strings into a chosen {@link FORMAT} and writes them into PubSub topic. */ public static class FormatOutput extends PTransform, PDone> { diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java index 7d97ed60f7af..3ca0b000a644 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java @@ -16,5 +16,5 @@ * limitations under the License. */ -/** Kafka to Pubsub template for Google Cloud Dataflow. */ +/** Kafka to Pubsub template. */ package org.apache.beam.templates.transforms; From f754aa81f0a307ab78c73266d576256da0ac8ecc Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 22 Oct 2020 23:45:56 +0300 Subject: [PATCH 35/73] Added ssl to AVRO reader --- .../java/org/apache/beam/templates/KafkaToPubsub.java | 5 +++-- .../beam/templates/options/KafkaToPubsubOptions.java | 9 +++++++++ .../beam/templates/transforms/FormatTransform.java | 10 ++++++---- .../src/main/resources/kafka_to_pubsub_metadata.json | 2 +- 4 files changed, 19 insertions(+), 7 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 15babdeed77e..74dff3b59da4 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -260,7 +260,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { .apply( "readAvrosFromKafka", FormatTransform.readAvrosFromKafka( - options.getBootstrapServers(), topicsList, kafkaConfig)) + options.getBootstrapServers(), topicsList, kafkaConfig, sslConfig)) .apply("createValues", Values.create()) .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); @@ -268,7 +268,8 @@ public static PipelineResult run(KafkaToPubsubOptions options) { pipeline .apply( "readFromKafka", - FormatTransform.readFromKafka(options.getBootstrapServers(), topicsList, kafkaConfig, sslConfig)) + FormatTransform.readFromKafka( + options.getBootstrapServers(), topicsList, kafkaConfig, sslConfig)) .apply("createValues", Values.create()) .apply("writeToPubSub", new FormatTransform.FormatOutput(options)); } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 339350cc4b09..53cea27eeb7d 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -17,9 +17,11 @@ */ package org.apache.beam.templates.options; +import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; +import org.apache.beam.templates.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { @Description("Kafka Bootstrap Servers") @@ -43,6 +45,13 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setOutputTopic(String outputTopic); + @Description("") + @Validation.Required + @Default.Enum("PUBSUB") + FormatTransform.FORMAT getOutputFormat(); + + void setOutputFormat(FormatTransform.FORMAT outputFormat); + @Description("URL to credentials in Vault") String getSecretStoreUrl(); diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index 3b4020ed060e..d0d980d8d47a 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.beam.templates.ConsumerFactoryFn; import org.apache.beam.templates.avro.TaxiRide; import org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer; -import org.apache.beam.templates.ConsumerFactoryFn; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -45,8 +45,7 @@ public class FormatTransform { public enum FORMAT { PUBSUB, - PLAINTEXT, - AVRO; + AVRO } /** @@ -84,7 +83,9 @@ public static PTransform>> readFromKafka( * @return configured reading from Kafka */ public static PTransform>> readAvrosFromKafka( - String bootstrapServers, List topicsList, Map config,Map kafkaConfig, + String bootstrapServers, + List topicsList, + Map config, Map sslConfig) { return KafkaIO.read() .withBootstrapServers(bootstrapServers) @@ -94,6 +95,7 @@ public static PTransform>> readAvrosFro .withValueDeserializerAndCoder( TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) .withConsumerConfigUpdates(config) + .withConsumerFactoryFn(new ConsumerFactoryFn(sslConfig)) .withoutMetadata(); } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index 1830234b4955..6abc82263c1b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -28,7 +28,7 @@ "label": "Message format that is written into Pub/Sub", "helpText": "One of two message formats that is written to Pub/Sub. PUBSUB or PLAINTEXT. Default is PLAINTEXT", "regexes": [ - "^(PUBSUB|PLAINTEXT)$" + "^(PUBSUB|AVRO)$" ], "isOptional": true }, From 695467aea548fa0a4c57cc0cac8db757d886a680 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 22 Oct 2020 23:51:57 +0300 Subject: [PATCH 36/73] FIX whitespaces. --- examples/templates/java/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 912d16a9af68..05d22b270832 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -32,11 +32,11 @@ Supported input source configurations: - Single or multiple Apache Kafka bootstrap servers - Apache Kafka SASL/SCRAM authentication over plaintext or SSL connection - Secrets vault service [HashiCorp Vault](https://www.vaultproject.io/). - + Supported destination configuration: - Single Google Pub/Sub topic. -In a simple scenario, the template will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The template supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the template will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. +In a simple scenario, the template will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The template supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the template will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. ## Requirements @@ -82,7 +82,7 @@ The result of the `shadowJar` task execution is a `.jar` file that is generated under the `build/libs/` folder in kafka-to-pubsub directory. ## Local execution -To execute this pipeline locally, specify the parameters: +To execute this pipeline locally, specify the parameters: - Kafka Bootstrap servers - Kafka input topics - Pub/Sub output topic @@ -92,7 +92,7 @@ in the following format: --inputTopics=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame ``` -Optionally, to retrieve Kafka credentials for SASL/SCRAM, +Optionally, to retrieve Kafka credentials for SASL/SCRAM, specify a URL to the credentials in HashiCorp Vault and the vault access token: ```bash --secretStoreUrl=http(s)://host:port/path/to/credentials @@ -155,7 +155,7 @@ Dataflow Flex Templates package the pipeline as a Docker image and stage these i on your project's [Container Registry](https://cloud.google.com/container-registry). To execute the template you need to create the template spec file containing all -the necessary information to run the job. This template already has the following +the necessary information to run the job. This template already has the following [metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: @@ -224,4 +224,4 @@ You can do this in 3 different ways: "${TEMPLATES_LAUNCH_API}" ``` -_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ +_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration. From 0212b90f22159e083f67fe595284fa4b76ce43f1 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Fri, 23 Oct 2020 17:52:51 +0300 Subject: [PATCH 37/73] added readme/docs regarding of Avro --- examples/templates/java/README.md | 25 +++++++++++++++++++ .../apache/beam/templates/KafkaToPubsub.java | 14 +++++++++++ 2 files changed, 39 insertions(+) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 05d22b270832..3d51ca94a396 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -56,6 +56,7 @@ This section describes what is needed to get the template up and running. - Set up the environment - Creating the Dataflow Flex Template - Create a Dataflow job to ingest data using the template. +- Avro format transferring. ## Assembling the Uber-JAR @@ -224,4 +225,28 @@ You can do this in 3 different ways: "${TEMPLATES_LAUNCH_API}" ``` +## AVRO format transferring. +In this template presented example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub. + +To use this example in the specific case, follow the few steps: + +- Create your own class to describe AVRO schema. As an example use [TaxiRide class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [TaxiRidesKafkaAvroDeserializer class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +```java +return KafkaIO.read() + ... + .withValueDeserializerAndCoder( + TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) // put your classes here + ... +``` +- Modify write step in the [KafkaToPubsub class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by put your Schema class to "writeAvrosToPubSub" step. +```java +if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { + ... + .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); // put your SCHEMA class here + + } +``` + _Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration. diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 74dff3b59da4..7d61e7b9f5de 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -123,6 +123,20 @@ * ' * "${TEMPLATES_LAUNCH_API}" *
  • + * + *

    Example Avro usage + * + *

    + * In this template presented example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub.
    + *
    + * To use this example in the specific case, follow the few steps:
    + * 
      + *
    • Create your own class to describe AVRO schema. As an example use {@link TaxiRide}. Just define necessary fields. + *
    • Create your own Avro Deserializer class. As an example use {@link org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer}. Just rename it, and put your own Schema class as the necessary types. + *
    • Modify the {@link FormatTransform}. Put your Schema class and Deserializer to the related parameter. + *
    • Modify write step in the {@link KafkaToPubsub} by put your Schema class to "writeAvrosToPubSub" step. + *
    + *
    */ public class KafkaToPubsub { From 0d0c8249e1143b5198509539b3cb313cea103c7a Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 26 Oct 2020 12:56:25 +0300 Subject: [PATCH 38/73] README.md and javadoc fixes --- examples/templates/java/README.md | 14 +++++++------- .../org/apache/beam/templates/KafkaToPubsub.java | 2 +- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 3d51ca94a396..b49214705e8f 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -226,13 +226,13 @@ You can do this in 3 different ways: ``` ## AVRO format transferring. -In this template presented example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub. +This template contains an example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub. -To use this example in the specific case, follow the few steps: +To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [TaxiRide class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [TaxiRidesKafkaAvroDeserializer class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [TaxiRide class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [TaxiRidesKafkaAvroDeserializer class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java return KafkaIO.read() ... @@ -240,7 +240,7 @@ return KafkaIO.read() TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) // put your classes here ... ``` -- Modify write step in the [KafkaToPubsub class](examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by put your Schema class to "writeAvrosToPubSub" step. +- Modify the write step in the [KafkaToPubsub class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { ... @@ -249,4 +249,4 @@ if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { } ``` -_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration. +_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 7d61e7b9f5de..84ed6b2b527b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -127,7 +127,7 @@ *

    Example Avro usage * *

    - * In this template presented example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub.
    + * This template contains an example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub.
      *
      * To use this example in the specific case, follow the few steps:
      * 
      From a4fea29c76c33321271f21b699e379e3b66a2edc Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 26 Oct 2020 17:52:07 +0300 Subject: [PATCH 39/73] Added Vault's response JSON schema description --- .../apache/beam/templates/KafkaToPubsub.java | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 84ed6b2b527b..a78ce17107ba 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -173,6 +173,32 @@ public static Map configureKafka(String secretStoreUrl, String t HttpResponse response = client.execute(request); String json = EntityUtils.toString(response.getEntity(), "UTF-8"); + /* + Vault's response JSON has a specific schema, where the actual data is placed under + {data: {data: }}. + Example: + { + "request_id": "6a0bb14b-ef24-256c-3edf-cfd52ad1d60d", + "lease_id": "", + "renewable": false, + "lease_duration": 0, + "data": { + "data": { + "password": "admin-secret", + "username": "admin" + }, + "metadata": { + "created_time": "2020-10-20T11:43:11.109186969Z", + "deletion_time": "", + "destroyed": false, + "version": 8 + } + }, + "wrap_info": null, + "warnings": null, + "auth": null + } + */ // Parse username and password from the response JSON JsonObject credentials = JsonParser.parseString(json) From 17957b8a9f5468883de469dd9999fbb051301beb Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 28 Oct 2020 16:10:59 +0300 Subject: [PATCH 40/73] Style fix --- .../apache/beam/templates/KafkaToPubsub.java | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index a78ce17107ba..5b1d450eceb0 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -174,31 +174,31 @@ public static Map configureKafka(String secretStoreUrl, String t String json = EntityUtils.toString(response.getEntity(), "UTF-8"); /* - Vault's response JSON has a specific schema, where the actual data is placed under - {data: {data: }}. - Example: - { - "request_id": "6a0bb14b-ef24-256c-3edf-cfd52ad1d60d", - "lease_id": "", - "renewable": false, - "lease_duration": 0, - "data": { - "data": { - "password": "admin-secret", - "username": "admin" - }, - "metadata": { - "created_time": "2020-10-20T11:43:11.109186969Z", - "deletion_time": "", - "destroyed": false, - "version": 8 - } - }, - "wrap_info": null, - "warnings": null, - "auth": null - } - */ + Vault's response JSON has a specific schema, where the actual data is placed under + {data: {data: }}. + Example: + { + "request_id": "6a0bb14b-ef24-256c-3edf-cfd52ad1d60d", + "lease_id": "", + "renewable": false, + "lease_duration": 0, + "data": { + "data": { + "password": "admin-secret", + "username": "admin" + }, + "metadata": { + "created_time": "2020-10-20T11:43:11.109186969Z", + "deletion_time": "", + "destroyed": false, + "version": 8 + } + }, + "wrap_info": null, + "warnings": null, + "auth": null + } + */ // Parse username and password from the response JSON JsonObject credentials = JsonParser.parseString(json) From b4e7081334f1f21e7e28e522c884099ecc34605e Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 19 Nov 2020 19:10:01 +0300 Subject: [PATCH 41/73] Refactoring. --- examples/templates/java/README.md | 2 +- .../java/kafka-to-pubsub/build.gradle | 2 - .../beam/templates/KafkaPubsubConstants.java | 29 +++ .../apache/beam/templates/KafkaToPubsub.java | 122 ++----------- .../consumer/SslConsumerFactoryFn.java} | 8 +- .../beam/templates/kafka/consumer/Utils.java | 169 ++++++++++++++++++ .../kafka/consumer/package-info.java | 20 +++ .../options/KafkaToPubsubOptions.java | 11 +- .../templates/transforms/FormatTransform.java | 6 +- 9 files changed, 243 insertions(+), 126 deletions(-) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java rename examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/{ConsumerFactoryFn.java => kafka/consumer/SslConsumerFactoryFn.java} (92%) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index b49214705e8f..721d030c3e5c 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -173,7 +173,7 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ --sdk-language "JAVA" \ --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ - --jar "build/libs/beam-examples-templates-java-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \ + --jar "build/libs/beam-examples-templates-java-kafka-to-pubsub--all.jar" \ --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" ``` diff --git a/examples/templates/java/kafka-to-pubsub/build.gradle b/examples/templates/java/kafka-to-pubsub/build.gradle index 50eb7d433e01..b23a8732c3aa 100644 --- a/examples/templates/java/kafka-to-pubsub/build.gradle +++ b/examples/templates/java/kafka-to-pubsub/build.gradle @@ -49,5 +49,3 @@ dependencies { testCompile group: 'junit', name: 'junit', version: '4.12' } -def gcpProject = project.findProperty('gcpProject') ?: 'apache-beam-testing' -def gcsTempRoot = project.findProperty('gcsTempRoot') ?: 'gs://temp-storage-for-end-to-end-tests/' diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java new file mode 100644 index 000000000000..07c55d8c7746 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java @@ -0,0 +1,29 @@ +/* + * 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.templates; + +/** Constant variables that are used across the template's parts. */ +public class KafkaPubsubConstants { + + /* Config keywords */ + public static final String KAFKA_CREDENTIALS = "kafka"; + public static final String SSL_CREDENTIALS = "ssl"; + public static final String USERNAME = "username"; + public static final String PASSWORD = "password"; + public static final String BUCKET = "bucket"; +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 5b1d450eceb0..576fae75d660 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -17,11 +17,10 @@ */ package org.apache.beam.templates; +import static org.apache.beam.templates.kafka.consumer.Utils.configureKafka; +import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.gson.JsonObject; -import com.google.gson.JsonParser; -import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; import java.util.HashMap; @@ -35,15 +34,6 @@ import org.apache.beam.templates.avro.TaxiRide; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.templates.transforms.FormatTransform; -import org.apache.http.HttpResponse; -import org.apache.http.client.HttpClient; -import org.apache.http.client.methods.HttpGet; -import org.apache.http.impl.client.HttpClientBuilder; -import org.apache.http.util.EntityUtils; -import org.apache.kafka.common.config.SaslConfigs; -import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.security.auth.SecurityProtocol; -import org.apache.kafka.common.security.scram.ScramMechanism; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -155,84 +145,6 @@ public static void main(String[] args) { run(options); } - /** - * Retrieves username and password from HashiCorp Vault secret storage and configures Kafka - * consumer for authorized connection. - * - * @param secretStoreUrl url to the secret storage that contains a credentials for Kafka - * @param token Vault token to access the secret storage - * @return configuration set of parameters for Kafka - * @throws IOException throws in case of the failure to execute the request to the secret storage - */ - public static Map configureKafka(String secretStoreUrl, String token) - throws IOException { - // Execute a request to get the credentials - HttpClient client = HttpClientBuilder.create().build(); - HttpGet request = new HttpGet(secretStoreUrl); - request.addHeader("X-Vault-Token", token); - HttpResponse response = client.execute(request); - String json = EntityUtils.toString(response.getEntity(), "UTF-8"); - - /* - Vault's response JSON has a specific schema, where the actual data is placed under - {data: {data: }}. - Example: - { - "request_id": "6a0bb14b-ef24-256c-3edf-cfd52ad1d60d", - "lease_id": "", - "renewable": false, - "lease_duration": 0, - "data": { - "data": { - "password": "admin-secret", - "username": "admin" - }, - "metadata": { - "created_time": "2020-10-20T11:43:11.109186969Z", - "deletion_time": "", - "destroyed": false, - "version": 8 - } - }, - "wrap_info": null, - "warnings": null, - "auth": null - } - */ - // Parse username and password from the response JSON - JsonObject credentials = - JsonParser.parseString(json) - .getAsJsonObject() - .get("data") - .getAsJsonObject() - .getAsJsonObject("data"); - String username = credentials.get("username").getAsString(); - String password = credentials.get("password").getAsString(); - - // Create the configuration for Kafka - Map config = new HashMap<>(); - config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_512.mechanismName()); - config.put("security.protocol", SecurityProtocol.SASL_PLAINTEXT.name()); - config.put( - SaslConfigs.SASL_JAAS_CONFIG, - String.format( - "org.apache.kafka.common.security.scram.ScramLoginModule required " - + "username=\"%s\" password=\"%s\";", - username, password)); - return config; - } - - public static Map configureSsl(KafkaToPubsubOptions options) { - Map config = new HashMap<>(); - config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, options.getTruststorePath()); - config.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, options.getKeystorePath()); - config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, options.getTruststorePassword()); - config.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, options.getKeystorePassword()); - config.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, options.getKeyPassword()); - - return config; - } - /** * Runs a pipeline which reads message from Kafka and writes it to GCS. * @@ -241,28 +153,16 @@ public static Map configureSsl(KafkaToPubsubOptions options) { public static PipelineResult run(KafkaToPubsubOptions options) { // Configure Kafka consumer properties Map kafkaConfig = new HashMap<>(); - try { - String secretStoreUrl = options.getSecretStoreUrl(); - String token = options.getVaultToken(); - kafkaConfig.putAll(configureKafka(secretStoreUrl, token)); - } catch (NullPointerException exception) { - LOG.info( + Map sslConfig = null; + if (options.getSecretStoreUrl() != null && options.getVaultToken() != null) { + Map> credentials = + getKafkaCredentialsFromVault(options.getSecretStoreUrl(), options.getVaultToken()); + kafkaConfig = configureKafka(credentials.get(KafkaPubsubConstants.KAFKA_CREDENTIALS)); + sslConfig = credentials.get(KafkaPubsubConstants.SSL_CREDENTIALS); + } else { + LOG.warn( "No information to retrieve Kafka credentials was provided. " + "Trying to initiate an unauthorized connection."); - } catch (IOException exception) { - LOG.error( - "Failed to retrieve credentials for Kafka client. " - + "Trying to initiate an unauthorized connection.", - exception); - } - - Map sslConfig = new HashMap<>(); - try { - sslConfig.putAll(configureSsl(options)); - } catch (NullPointerException e) { - LOG.info( - "No information to retrieve SSL certificate was provided. " - + "Trying to initiate a plain text connection."); } List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); @@ -304,7 +204,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { .apply("createValues", Values.create()) .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); - } else { + } else if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { pipeline .apply( "readFromKafka", diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java similarity index 92% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java index ff3ecee353f7..144787674b46 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/ConsumerFactoryFn.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates; +package org.apache.beam.templates.kafka.consumer; import java.io.File; import java.io.IOException; @@ -30,14 +30,14 @@ import org.slf4j.LoggerFactory; /** Class to create Kafka Consumer with configured SSL. */ -public class ConsumerFactoryFn +public class SslConsumerFactoryFn implements SerializableFunction, Consumer> { private final Map sslConfig; /* Logger for class.*/ - private static final Logger LOG = LoggerFactory.getLogger(ConsumerFactoryFn.class); + private static final Logger LOG = LoggerFactory.getLogger(SslConsumerFactoryFn.class); - public ConsumerFactoryFn(Map sslConfig) { + public SslConsumerFactoryFn(Map sslConfig) { this.sslConfig = sslConfig; } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java new file mode 100644 index 000000000000..b82119e26cda --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java @@ -0,0 +1,169 @@ +/* + * 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.templates.kafka.consumer; + +import static org.apache.beam.templates.KafkaPubsubConstants.BUCKET; +import static org.apache.beam.templates.KafkaPubsubConstants.KAFKA_CREDENTIALS; +import static org.apache.beam.templates.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.templates.KafkaPubsubConstants.SSL_CREDENTIALS; +import static org.apache.beam.templates.KafkaPubsubConstants.USERNAME; + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; +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.http.HttpResponse; +import org.apache.http.client.HttpClient; +import org.apache.http.client.methods.HttpGet; +import org.apache.http.impl.client.HttpClientBuilder; +import org.apache.http.util.EntityUtils; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.common.security.scram.ScramMechanism; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** Utilities for construction of Kafka Consumer. */ +public class Utils { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(Utils.class); + + /** + * Retrieves all credentials from HashiCorp Vault secret storage. + * + * @param secretStoreUrl url to the secret storage that contains a credentials for Kafka + * @param token Vault token to access the secret storage + * @return credentials for Kafka consumer config + */ + public static Map> getKafkaCredentialsFromVault( + String secretStoreUrl, String token) { + Map> credentialMap = new HashMap<>(); + + JsonObject credentials = null; + try { + HttpClient client = HttpClientBuilder.create().build(); + HttpGet request = new HttpGet(secretStoreUrl); + request.addHeader("X-Vault-Token", token); + HttpResponse response = client.execute(request); + String json = EntityUtils.toString(response.getEntity(), "UTF-8"); + + /* + Vault's response JSON has a specific schema, where the actual data is placed under + {data: {data: }}. + Example: + { + "request_id": "6a0bb14b-ef24-256c-3edf-cfd52ad1d60d", + "lease_id": "", + "renewable": false, + "lease_duration": 0, + "data": { + "data": { + "bucket": "kafka_to_pubsub_test", + "key_password": "secret", + "keystore_password": "secret", + "keystore_path": "ssl_cert/kafka.keystore.jks", + "password": "admin-secret", + "truststore_password": "secret", + "truststore_path": "ssl_cert/kafka.truststore.jks", + "username": "admin" + }, + "metadata": { + "created_time": "2020-10-20T11:43:11.109186969Z", + "deletion_time": "", + "destroyed": false, + "version": 8 + } + }, + "wrap_info": null, + "warnings": null, + "auth": null + } + */ + // Parse security properties from the response JSON + credentials = + JsonParser.parseString(json) + .getAsJsonObject() + .get("data") + .getAsJsonObject() + .getAsJsonObject("data"); + } catch (IOException e) { + LOG.error("Failed to retrieve credentials from Vault.", e); + } + + if (credentials != null) { + // Username and password for Kafka authorization + credentialMap.put(KAFKA_CREDENTIALS, new HashMap<>()); + + if (credentials.has(USERNAME) && credentials.has(PASSWORD)) { + credentialMap.get(KAFKA_CREDENTIALS).put(USERNAME, credentials.get(USERNAME).getAsString()); + credentialMap.get(KAFKA_CREDENTIALS).put(PASSWORD, credentials.get(PASSWORD).getAsString()); + } else { + LOG.warn( + "There are no username and/or password for Kafka in Vault." + + "Trying to initiate an unauthorized connection."); + } + + // SSL truststore, keystore, and password + try { + Map sslCredentials = new HashMap<>(); + String[] configNames = { + BUCKET, + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, + SslConfigs.SSL_KEY_PASSWORD_CONFIG + }; + for (String configName : configNames) { + sslCredentials.put(configName, credentials.get(configName).getAsString()); + } + credentialMap.put(SSL_CREDENTIALS, sslCredentials); + } catch (NullPointerException e) { + LOG.warn( + "There is no enough information to configure SSL." + + "Trying to initiate an unsecure connection.", + e); + } + } + + return credentialMap; + } + + /** + * Configures Kafka consumer for authorized connection. + * + * @param props username and password for Kafka + * @return configuration set of parameters for Kafka + */ + public static Map configureKafka(Map props) { + // Create the configuration for Kafka + Map config = new HashMap<>(); + if (props != null && props.containsKey(USERNAME) && props.containsKey(PASSWORD)) { + config.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_512.mechanismName()); + config.put( + SaslConfigs.SASL_JAAS_CONFIG, + String.format( + "org.apache.kafka.common.security.scram.ScramLoginModule required " + + "username=\"%s\" password=\"%s\";", + props.get(USERNAME), props.get(PASSWORD))); + } + return config; + } +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java new file mode 100644 index 000000000000..43df3c742fd1 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template. */ +package org.apache.beam.templates.kafka.consumer; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 53cea27eeb7d..01cbc9b12ac3 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -17,20 +17,21 @@ */ package org.apache.beam.templates.options; -import org.apache.beam.sdk.options.Default; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; import org.apache.beam.templates.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { - @Description("Kafka Bootstrap Servers") + @Description( + "Comma Separated list of Kafka Bootstrap Servers (e.g: server1:[port],server2:[port]).") @Validation.Required String getBootstrapServers(); void setBootstrapServers(String value); - @Description("Kafka topics to read the input from") + @Description( + "Comma Separated list of Kafka topic(s) to read the input from (e.g: topic1,topic2).") @Validation.Required String getInputTopics(); @@ -45,9 +46,9 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setOutputTopic(String outputTopic); - @Description("") + @Description( + "Format which will be writen to output Pub/Sub topic. Supported formats: AVRO, PUBSUB") @Validation.Required - @Default.Enum("PUBSUB") FormatTransform.FORMAT getOutputFormat(); void setOutputFormat(FormatTransform.FORMAT outputFormat); diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index d0d980d8d47a..35a6efd07493 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -32,9 +32,9 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.templates.ConsumerFactoryFn; import org.apache.beam.templates.avro.TaxiRide; import org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer; +import org.apache.beam.templates.kafka.consumer.SslConsumerFactoryFn; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; @@ -70,7 +70,7 @@ public static PTransform>> readFromKafka( .withValueDeserializerAndCoder( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) .withConsumerConfigUpdates(kafkaConfig) - .withConsumerFactoryFn(new ConsumerFactoryFn(sslConfig)) + .withConsumerFactoryFn(new SslConsumerFactoryFn(sslConfig)) .withoutMetadata(); } @@ -95,7 +95,7 @@ public static PTransform>> readAvrosFro .withValueDeserializerAndCoder( TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) .withConsumerConfigUpdates(config) - .withConsumerFactoryFn(new ConsumerFactoryFn(sslConfig)) + .withConsumerFactoryFn(new SslConsumerFactoryFn(sslConfig)) .withoutMetadata(); } From aa857d7b314ade00e5e387cbf7cc8dca7b8ef649 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 19 Nov 2020 19:53:58 +0300 Subject: [PATCH 42/73] Fixed ssl parameters --- .../apache/beam/templates/KafkaToPubsub.java | 14 ++++-- .../beam/templates/kafka/consumer/Utils.java | 43 ++++++++----------- .../options/KafkaToPubsubOptions.java | 4 +- .../resources/kafka_to_pubsub_metadata.json | 32 ++++++++++++++ 4 files changed, 63 insertions(+), 30 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 576fae75d660..0a54eefb1c44 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -17,8 +17,7 @@ */ package org.apache.beam.templates; -import static org.apache.beam.templates.kafka.consumer.Utils.configureKafka; -import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; +import static org.apache.beam.templates.kafka.consumer.Utils.*; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayList; @@ -153,18 +152,25 @@ public static void main(String[] args) { public static PipelineResult run(KafkaToPubsubOptions options) { // Configure Kafka consumer properties Map kafkaConfig = new HashMap<>(); - Map sslConfig = null; if (options.getSecretStoreUrl() != null && options.getVaultToken() != null) { Map> credentials = getKafkaCredentialsFromVault(options.getSecretStoreUrl(), options.getVaultToken()); kafkaConfig = configureKafka(credentials.get(KafkaPubsubConstants.KAFKA_CREDENTIALS)); - sslConfig = credentials.get(KafkaPubsubConstants.SSL_CREDENTIALS); } else { LOG.warn( "No information to retrieve Kafka credentials was provided. " + "Trying to initiate an unauthorized connection."); } + Map sslConfig = new HashMap<>(); + if (isSslSpecified(options)) { + sslConfig.putAll(configureSsl(options)); + } else { + LOG.info( + "No information to retrieve SSL certificate was provided. " + + "Trying to initiate a plain text connection."); + } + List topicsList = new ArrayList<>(Arrays.asList(options.getInputTopics().split(","))); checkArgument( diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java index b82119e26cda..49a8b46be6c1 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java @@ -17,15 +17,14 @@ */ package org.apache.beam.templates.kafka.consumer; -import static org.apache.beam.templates.KafkaPubsubConstants.BUCKET; import static org.apache.beam.templates.KafkaPubsubConstants.KAFKA_CREDENTIALS; import static org.apache.beam.templates.KafkaPubsubConstants.PASSWORD; -import static org.apache.beam.templates.KafkaPubsubConstants.SSL_CREDENTIALS; import static org.apache.beam.templates.KafkaPubsubConstants.USERNAME; import java.io.IOException; import java.util.HashMap; import java.util.Map; +import org.apache.beam.templates.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.http.HttpResponse; @@ -119,28 +118,6 @@ public static Map> getKafkaCredentialsFromVault( "There are no username and/or password for Kafka in Vault." + "Trying to initiate an unauthorized connection."); } - - // SSL truststore, keystore, and password - try { - Map sslCredentials = new HashMap<>(); - String[] configNames = { - BUCKET, - SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, - SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, - SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, - SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, - SslConfigs.SSL_KEY_PASSWORD_CONFIG - }; - for (String configName : configNames) { - sslCredentials.put(configName, credentials.get(configName).getAsString()); - } - credentialMap.put(SSL_CREDENTIALS, sslCredentials); - } catch (NullPointerException e) { - LOG.warn( - "There is no enough information to configure SSL." - + "Trying to initiate an unsecure connection.", - e); - } } return credentialMap; @@ -166,4 +143,22 @@ public static Map configureKafka(Map props) { } return config; } + + public static Map configureSsl(KafkaToPubsubOptions options) { + Map config = new HashMap<>(); + config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, options.getTruststorePath()); + config.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, options.getKeystorePath()); + config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, options.getTruststorePassword()); + config.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, options.getKeystorePassword()); + config.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, options.getKeyPassword()); + + return config; + } + + public static boolean isSslSpecified(KafkaToPubsubOptions options) { + return options.getTruststorePath() != null + || options.getTruststorePassword() != null + || options.getKeystorePath() != null + || options.getKeyPassword() != null; + } } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java index 01cbc9b12ac3..92b24636dcc7 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java @@ -73,12 +73,12 @@ public interface KafkaToPubsubOptions extends PipelineOptions { void setKeystorePath(String keystorePath); - @Description("The password for the trust store file") + @Description("The password for the trust store password") String getTruststorePassword(); void setTruststorePassword(String truststorePassword); - @Description("The store password for the key store file") + @Description("The store password for the key store password") String getKeystorePassword(); void setKeystorePassword(String keystorePassword); diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index 6abc82263c1b..65469c4bf78f 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -47,6 +47,38 @@ "regexes": [], "paramType": "TEXT", "isOptional": true + }, + { + "name": "truststorePath", + "label": "Truststore Path", + "helpText": "The path to the trust store file", + "regexes": [], + "paramType": "TEXT", + "isOptional": true + }, + { + "name": "truststorePassword", + "label": "The password for the trust store password", + "helpText": "Token to access HashiCorp Vault secret storage", + "regexes": [], + "paramType": "TEXT", + "isOptional": true + }, + { + "name": "keystorePath", + "label": "The path to the key store file", + "helpText": "Token to access HashiCorp Vault secret storage", + "regexes": [], + "paramType": "TEXT", + "isOptional": true + }, + { + "name": "keystorePassword", + "label": "Keystore Password", + "helpText": "The store password for the key store password", + "regexes": [], + "paramType": "TEXT", + "isOptional": true } ] } From 25efe95508dedcacd9374fffd67e9f0d40e7b51e Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Thu, 19 Nov 2020 19:57:19 +0300 Subject: [PATCH 43/73] Fixed style --- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 0a54eefb1c44..e338fbca089b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -17,7 +17,10 @@ */ package org.apache.beam.templates; -import static org.apache.beam.templates.kafka.consumer.Utils.*; +import static org.apache.beam.templates.kafka.consumer.Utils.configureKafka; +import static org.apache.beam.templates.kafka.consumer.Utils.configureSsl; +import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; +import static org.apache.beam.templates.kafka.consumer.Utils.isSslSpecified; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayList; From f300de51ca3b07d575002b3e9a14970911aa5ce0 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Fri, 20 Nov 2020 18:12:39 +0300 Subject: [PATCH 44/73] optimize build.gradle --- examples/templates/java/kafka-to-pubsub/build.gradle | 2 -- 1 file changed, 2 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/build.gradle b/examples/templates/java/kafka-to-pubsub/build.gradle index b23a8732c3aa..d311b10c0251 100644 --- a/examples/templates/java/kafka-to-pubsub/build.gradle +++ b/examples/templates/java/kafka-to-pubsub/build.gradle @@ -23,8 +23,6 @@ plugins { } applyJavaNature( exportJavadoc: false, - enableChecker: false, - ignoreRawtypeErrors: true, automaticModuleName: 'org.apache.beam.examples.templates.kafka-to-pubsub') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() From c0e6ad0d5943795839635b977cf1a875680ef2d9 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Fri, 20 Nov 2020 18:31:43 +0300 Subject: [PATCH 45/73] Resolve conversations --- examples/templates/java/README.md | 6 ++-- examples/templates/java/build.gradle | 31 ------------------- .../apache/beam/templates/avro/TaxiRide.java | 2 -- 3 files changed, 4 insertions(+), 35 deletions(-) delete mode 100644 examples/templates/java/build.gradle diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index 721d030c3e5c..d32c3df91b1e 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -22,7 +22,7 @@ This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic -in [Google Pub/Sub](https://cloud.google.com/pubsub). +in [Google Cloud Pub/Sub](https://cloud.google.com/pubsub). Supported data formats: - Serializable plaintext formats, such as JSON @@ -34,7 +34,7 @@ Supported input source configurations: - Secrets vault service [HashiCorp Vault](https://www.vaultproject.io/). Supported destination configuration: -- Single Google Pub/Sub topic. +- Single Google Cloud Pub/Sub topic. In a simple scenario, the template will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The template supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the template will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. @@ -183,6 +183,8 @@ To deploy the pipeline, you should refer to the template file and pass the [parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) required by the pipeline. +To understand how works flex templates, please take a look at this [Documentation](https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates) + You can do this in 3 different ways: 1. Using [Dataflow Google Cloud Console](https://console.cloud.google.com/dataflow/jobs) diff --git a/examples/templates/java/build.gradle b/examples/templates/java/build.gradle deleted file mode 100644 index f6934dbe6c72..000000000000 --- a/examples/templates/java/build.gradle +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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. - */ - -plugins { - id 'java' -} - -version '2.25.0-SNAPSHOT' - -repositories { - mavenCentral() -} - -dependencies { - testCompile group: 'junit', name: 'junit', version: '4.12' -} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java index c669cffcdd33..2f951c63b752 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java @@ -43,8 +43,6 @@ public class TaxiRide { String rideStatus; Integer passengerCount; - public TaxiRide() {} - public TaxiRide( String rideId, Float latitude, From 998f4c0f5d1631e157cb61fca02908e7bdad2a33 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 23 Nov 2020 13:07:24 +0300 Subject: [PATCH 46/73] Updated regarding comments and added unit tests --- .../apache/beam/templates/KafkaToPubsub.java | 2 +- .../kafka/consumer/SslConsumerFactoryFn.java | 49 +++++++++++- .../beam/templates/kafka/consumer/Utils.java | 3 +- .../templates/transforms/FormatTransform.java | 6 +- .../beam/templates/KafkaToPubsubTest.java | 78 ++++++++++++++----- 5 files changed, 109 insertions(+), 29 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index e338fbca089b..77e1b86d44c8 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -88,7 +88,7 @@ * --sdk-language "JAVA" \ * --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ * --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ - * --jar "build/libs/beam-templates-kafka-to-pubsub-2.25.0-SNAPSHOT-all.jar" \ + * --jar "build/libs/beam-templates-kafka-to-pubsub--all.jar" \ * --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" * * # Execute template: diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java index 144787674b46..6b6b29e0088f 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java @@ -19,7 +19,14 @@ import java.io.File; import java.io.IOException; +import java.nio.channels.FileChannel; +import java.nio.channels.ReadableByteChannel; +import java.nio.file.Paths; +import java.nio.file.StandardOpenOption; +import java.util.HashSet; import java.util.Map; +import java.util.Set; +import org.apache.beam.sdk.io.FileSystems; import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.kafka.clients.CommonClientConfigs; import org.apache.kafka.clients.consumer.Consumer; @@ -33,6 +40,8 @@ public class SslConsumerFactoryFn implements SerializableFunction, Consumer> { private final Map sslConfig; + private static final String TRUSTSTORE_LOCAL_PATH = "/tmp/kafka.truststore.jks"; + private static final String KEYSTORE_LOCAL_PATH = "/tmp/kafka.keystore.jks"; /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(SslConsumerFactoryFn.class); @@ -41,12 +50,24 @@ public SslConsumerFactoryFn(Map sslConfig) { this.sslConfig = sslConfig; } + @SuppressWarnings("nullness") @Override public Consumer apply(Map config) { try { - checkFileExists(sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); - checkFileExists(sslConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)); - } catch (IOException | NullPointerException e) { + String truststoreLocation = sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); + if (truststoreLocation.startsWith("gs://")) { + getGcsFileAsLocal(truststoreLocation, TRUSTSTORE_LOCAL_PATH); + } else { + checkFileExists(truststoreLocation); + } + + String keystoreLocation = sslConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); + if (keystoreLocation.startsWith("gs://")) { + getGcsFileAsLocal(keystoreLocation, KEYSTORE_LOCAL_PATH); + } else { + checkFileExists(keystoreLocation); + } + } catch (IOException e) { LOG.error("Failed to retrieve data for SSL", e); return new KafkaConsumer<>(config); } @@ -79,4 +100,26 @@ private void checkFileExists(String filePath) throws IOException { throw new IOException(); } } + + /** + * Reads a file from GCS and writes it locally. + * + * @param gcsFilePath path to file in GCS in format "gs://your-bucket/path/to/file" + * @param outputFilePath path where to save file locally + * @throws IOException thrown if not able to read or write file + */ + public static void getGcsFileAsLocal(String gcsFilePath, String outputFilePath) + throws IOException { + LOG.info("Reading contents from GCS file: {}", gcsFilePath); + Set options = new HashSet<>(2); + options.add(StandardOpenOption.CREATE); + options.add(StandardOpenOption.APPEND); + // Copy the GCS file into a local file and will throw an I/O exception in case file not found. + try (ReadableByteChannel readerChannel = + FileSystems.open(FileSystems.matchSingleFileSpec(gcsFilePath).resourceId())) { + try (FileChannel writeChannel = FileChannel.open(Paths.get(outputFilePath), options)) { + writeChannel.transferFrom(readerChannel, 0, Long.MAX_VALUE); + } + } + } } diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java index 49a8b46be6c1..5b87750651c7 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.security.scram.ScramMechanism; +import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -129,7 +130,7 @@ public static Map> getKafkaCredentialsFromVault( * @param props username and password for Kafka * @return configuration set of parameters for Kafka */ - public static Map configureKafka(Map props) { + public static Map configureKafka(@Nullable Map props) { // Create the configuration for Kafka Map config = new HashMap<>(); if (props != null && props.containsKey(USERNAME) && props.containsKey(PASSWORD)) { diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index 35a6efd07493..ab2deedd3cf9 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -66,9 +66,9 @@ public static PTransform>> readFromKafka( .withBootstrapServers(bootstrapServers) .withTopics(topicsList) .withKeyDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of()).getValueCoder()) .withValueDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of()).getValueCoder()) .withConsumerConfigUpdates(kafkaConfig) .withConsumerFactoryFn(new SslConsumerFactoryFn(sslConfig)) .withoutMetadata(); @@ -91,7 +91,7 @@ public static PTransform>> readAvrosFro .withBootstrapServers(bootstrapServers) .withTopics(topicsList) .withKeyDeserializerAndCoder( - StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of())) + StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of()).getValueCoder()) .withValueDeserializerAndCoder( TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) .withConsumerConfigUpdates(config) diff --git a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java index 35e496b39850..81aa9f0bf534 100644 --- a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java +++ b/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java @@ -17,16 +17,17 @@ */ package org.apache.beam.templates; -import java.util.ArrayList; -import java.util.Collections; +import static org.apache.beam.templates.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.templates.KafkaPubsubConstants.USERNAME; +import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; + import java.util.HashMap; -import java.util.List; -import org.apache.beam.sdk.Pipeline; -import org.apache.beam.sdk.testing.TestPipeline; -import org.apache.beam.templates.transforms.FormatTransform; -import org.junit.Rule; +import java.util.Map; +import org.apache.beam.templates.kafka.consumer.Utils; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.security.scram.ScramMechanism; +import org.junit.Assert; import org.junit.Test; -import org.junit.rules.ExpectedException; import org.junit.runner.RunWith; import org.junit.runners.JUnit4; @@ -34,21 +35,56 @@ @RunWith(JUnit4.class) public class KafkaToPubsubTest { - @Rule public final transient TestPipeline pipeline = TestPipeline.create(); + /** Tests configureKafka() with a null input properties. */ + @Test + public void testConfigureKafkaNullProps() { + Map config = Utils.configureKafka(null); + Assert.assertEquals(new HashMap<>(), config); + } - @Rule public ExpectedException thrown = ExpectedException.none(); + /** Tests configureKafka() without a Password in input properties. */ + @Test + public void testConfigureKafkaNoPassword() { + Map props = new HashMap<>(); + props.put(USERNAME, "username"); + Map config = Utils.configureKafka(props); + Assert.assertEquals(new HashMap<>(), config); + } + + /** Tests configureKafka() without a Username in input properties. */ + @Test + public void testConfigureKafkaNoUsername() { + Map props = new HashMap<>(); + props.put(PASSWORD, "password"); + Map config = Utils.configureKafka(props); + Assert.assertEquals(new HashMap<>(), config); + } + + /** Tests configureKafka() with an appropriate input properties. */ + @Test + public void testConfigureKafka() { + Map props = new HashMap<>(); + props.put(USERNAME, "username"); + props.put(PASSWORD, "password"); + + Map expectedConfig = new HashMap<>(); + expectedConfig.put(SaslConfigs.SASL_MECHANISM, ScramMechanism.SCRAM_SHA_512.mechanismName()); + expectedConfig.put( + SaslConfigs.SASL_JAAS_CONFIG, + String.format( + "org.apache.kafka.common.security.scram.ScramLoginModule required " + + "username=\"%s\" password=\"%s\";", + props.get(USERNAME), props.get(PASSWORD))); + + Map config = Utils.configureKafka(props); + Assert.assertEquals(expectedConfig, config); + } + /** Tests getKafkaCredentialsFromVault() with an invalid url. */ @Test - public void testKafkaReadingFailsWrongBootstrapServer() { - final String bootstrapServers = "some-server:9092"; - final List topicsList = new ArrayList<>(Collections.singletonList("TEST-TOPIC")); - final HashMap kafkaConfig = new HashMap<>(); - final HashMap sslConfig = new HashMap<>(); - - pipeline.apply( - FormatTransform.readFromKafka(bootstrapServers, topicsList, kafkaConfig, sslConfig)); - thrown.expect(Pipeline.PipelineExecutionException.class); - thrown.expectMessage("Failed to construct kafka consumer"); - pipeline.run(); + public void testGetKafkaCredentialsFromVaultInvalidUrl() { + Map> credentials = + getKafkaCredentialsFromVault("some-url", "some-token"); + Assert.assertEquals(new HashMap<>(), credentials); } } From 6a6aa46a96838ad3a989fee61adba8902f3023d3 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 23 Nov 2020 14:53:41 +0300 Subject: [PATCH 47/73] README.md update --- examples/templates/java/README.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index d32c3df91b1e..e30c1055f7c0 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -40,7 +40,7 @@ In a simple scenario, the template will create an Apache Beam pipeline that will ## Requirements -- Java 11 +- Java 8 - Kafka Bootstrap Server(s) up and running - Existing source Kafka topic(s) - An existing Pub/Sub destination output topic @@ -101,8 +101,8 @@ specify a URL to the credentials in HashiCorp Vault and the vault access token: ``` Optionally, to configure secure SSL connection between the Beam pipeline and Kafka, specify the parameters: -- A local path to a truststore file -- A local path to a keystore file +- A path to a truststore file (it can be a local path or a GCS path, which should start with `gs://`) +- A path to a keystore file (it can be a local path or a GCS path, which should start with `gs://`) - Truststore password - Keystore password - Key password @@ -251,4 +251,4 @@ if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { } ``` -_Note: The Kafka to Pub/Sub Dataflow Flex template doesn't support SSL configuration._ +_Note: The Kafka to Pub/Sub Dataflow Flex template supports SSL configuration with the certificate located only in GCS._ From 0858e4717809d44ddb5ca4fa1924fa55b9f85ce2 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 23 Nov 2020 16:37:37 +0300 Subject: [PATCH 48/73] made Avro class more abstract --- examples/templates/java/README.md | 10 +- .../apache/beam/templates/KafkaToPubsub.java | 9 +- .../beam/templates/avro/AvroDataClass.java | 68 ++++++++++ ...> AvroDataClassKafkaAvroDeserializer.java} | 8 +- .../apache/beam/templates/avro/TaxiRide.java | 128 ------------------ .../templates/transforms/FormatTransform.java | 12 +- 6 files changed, 88 insertions(+), 147 deletions(-) create mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java rename examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/{TaxiRidesKafkaAvroDeserializer.java => AvroDataClassKafkaAvroDeserializer.java} (84%) delete mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java diff --git a/examples/templates/java/README.md b/examples/templates/java/README.md index e30c1055f7c0..097a6c41902e 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/README.md @@ -232,21 +232,21 @@ This template contains an example Class to deserialize AVRO from Kafka and seria To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [TaxiRide class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [TaxiRidesKafkaAvroDeserializer class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. - Modify the [FormatTransform.readAvrosFromKafka method](kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java -return KafkaIO.read() +return KafkaIO.read() ... .withValueDeserializerAndCoder( - TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) // put your classes here + AvroDataClassKafkaAvroDeserializer.class, AvroCoder.of(AvroDataClass.class)) // put your classes here ... ``` - Modify the write step in the [KafkaToPubsub class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { ... - .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); // put your SCHEMA class here + .apply("writeAvrosToPubSub", PubsubIO.writeAvros(AvroDataClass.class)); // put your SCHEMA class here } ``` diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 77e1b86d44c8..33a3b54bedd5 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -33,7 +33,8 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.templates.avro.TaxiRide; +import org.apache.beam.templates.avro.AvroDataClass; +import org.apache.beam.templates.avro.AvroDataClassKafkaAvroDeserializer; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.templates.transforms.FormatTransform; import org.slf4j.Logger; @@ -123,8 +124,8 @@ * * To use this example in the specific case, follow the few steps: *
        - *
      • Create your own class to describe AVRO schema. As an example use {@link TaxiRide}. Just define necessary fields. - *
      • Create your own Avro Deserializer class. As an example use {@link org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer}. Just rename it, and put your own Schema class as the necessary types. + *
      • Create your own class to describe AVRO schema. As an example use {@link AvroDataClass}. Just define necessary fields. + *
      • Create your own Avro Deserializer class. As an example use {@link AvroDataClassKafkaAvroDeserializer}. Just rename it, and put your own Schema class as the necessary types. *
      • Modify the {@link FormatTransform}. Put your Schema class and Deserializer to the related parameter. *
      • Modify write step in the {@link KafkaToPubsub} by put your Schema class to "writeAvrosToPubSub" step. *
      @@ -211,7 +212,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { FormatTransform.readAvrosFromKafka( options.getBootstrapServers(), topicsList, kafkaConfig, sslConfig)) .apply("createValues", Values.create()) - .apply("writeAvrosToPubSub", PubsubIO.writeAvros(TaxiRide.class)); + .apply("writeAvrosToPubSub", PubsubIO.writeAvros(AvroDataClass.class)); } else if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { pipeline diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java new file mode 100644 index 000000000000..c3daedfa1ee5 --- /dev/null +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java @@ -0,0 +1,68 @@ +/* + * 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.templates.avro; + +import org.apache.beam.sdk.coders.AvroCoder; +import org.apache.beam.sdk.coders.DefaultCoder; + +/** + * Example of AVRO serialization class. To configure your AVRO schema, change this class to + * requirement schema definition + */ +@DefaultCoder(AvroCoder.class) +public class AvroDataClass { + + String field1; + Float field2; + Float field3; + + public AvroDataClass( + String field1, + Float field2, + Float field3 + ) { + this.field1 = field1; + this.field2 = field2; + this.field3 = field3; + + } + + public String getField1() { + return field1; + } + + public void setField1(String field1) { + this.field1 = field1; + } + + public Float getField2() { + return field2; + } + + public void setField2(Float field2) { + this.field2 = field2; + } + + public Float getField3() { + return field3; + } + + public void setField3(Float field3) { + this.field3 = field3; + } +} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java similarity index 84% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java rename to examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java index 474c032bb1d4..119a2162aaa8 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRidesKafkaAvroDeserializer.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java @@ -23,8 +23,8 @@ import org.apache.kafka.common.serialization.Deserializer; /** Example of custom AVRO Deserialize. */ -public class TaxiRidesKafkaAvroDeserializer extends AbstractKafkaAvroDeserializer - implements Deserializer { +public class AvroDataClassKafkaAvroDeserializer extends AbstractKafkaAvroDeserializer + implements Deserializer { @Override public void configure(Map configs, boolean isKey) { @@ -32,8 +32,8 @@ public void configure(Map configs, boolean isKey) { } @Override - public TaxiRide deserialize(String s, byte[] bytes) { - return (TaxiRide) this.deserialize(bytes); + public AvroDataClass deserialize(String s, byte[] bytes) { + return (AvroDataClass) this.deserialize(bytes); } @Override diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java deleted file mode 100644 index 2f951c63b752..000000000000 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/TaxiRide.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * 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.templates.avro; - -import org.apache.beam.sdk.coders.AvroCoder; -import org.apache.beam.sdk.coders.DefaultCoder; - -/** - * Example of AVRO serialization class. To configure your AVRO schema, change this class to - * requirement schema definition - */ -@DefaultCoder(AvroCoder.class) -public class TaxiRide { - // "ride_id":"a60ba4d8-1501-4b5b-93ee-b7864304d0e0", - // "latitude":40.66684000000033, - // "longitude":-73.83933000000202, - // "timestamp":"2016-08-31T11:04:02.025396463-04:00", - // "meter_reading":14.270274, - // "meter_increment":0.019336415, - // "ride_status":"enroute" / "pickup" / "dropoff" - // "passenger_count":2 - String rideId; - Float latitude; - Float longitude; - String timestamp; - Float meterReading; - Float meterIncrement; - String rideStatus; - Integer passengerCount; - - public TaxiRide( - String rideId, - Float latitude, - Float longitude, - String timestamp, - Float meterReading, - Float meterIncrement, - String rideStatus, - Integer passengerCount) { - this.rideId = rideId; - this.latitude = latitude; - this.longitude = longitude; - this.timestamp = timestamp; - this.meterReading = meterReading; - this.meterIncrement = meterIncrement; - this.rideStatus = rideStatus; - this.passengerCount = passengerCount; - } - - public String getRideId() { - return rideId; - } - - public void setRideId(String rideId) { - this.rideId = rideId; - } - - public Float getLatitude() { - return latitude; - } - - public void setLatitude(Float latitude) { - this.latitude = latitude; - } - - public Float getLongitude() { - return longitude; - } - - public void setLongitude(Float longitude) { - this.longitude = longitude; - } - - public String getTimestamp() { - return timestamp; - } - - public void setTimestamp(String timestamp) { - this.timestamp = timestamp; - } - - public Float getMeterReading() { - return meterReading; - } - - public void setMeterReading(Float meterReading) { - this.meterReading = meterReading; - } - - public Float getMeterIncrement() { - return meterIncrement; - } - - public void setMeterIncrement(Float meterIncrement) { - this.meterIncrement = meterIncrement; - } - - public String getRideStatus() { - return rideStatus; - } - - public void setRideStatus(String rideStatus) { - this.rideStatus = rideStatus; - } - - public Integer getPassengerCount() { - return passengerCount; - } - - public void setPassengerCount(Integer passengerCount) { - this.passengerCount = passengerCount; - } -} diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java index ab2deedd3cf9..fbb67044ddf9 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java @@ -32,8 +32,8 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.templates.avro.TaxiRide; -import org.apache.beam.templates.avro.TaxiRidesKafkaAvroDeserializer; +import org.apache.beam.templates.avro.AvroDataClass; +import org.apache.beam.templates.avro.AvroDataClassKafkaAvroDeserializer; import org.apache.beam.templates.kafka.consumer.SslConsumerFactoryFn; import org.apache.beam.templates.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; @@ -75,25 +75,25 @@ public static PTransform>> readFromKafka( } /** - * Configures Kafka consumer to read avros to {@link TaxiRide} format. + * Configures Kafka consumer to read avros to {@link AvroDataClass} format. * * @param bootstrapServers Kafka servers to read from * @param topicsList Kafka topics to read from * @param config configuration for the Kafka consumer * @return configured reading from Kafka */ - public static PTransform>> readAvrosFromKafka( + public static PTransform>> readAvrosFromKafka( String bootstrapServers, List topicsList, Map config, Map sslConfig) { - return KafkaIO.read() + return KafkaIO.read() .withBootstrapServers(bootstrapServers) .withTopics(topicsList) .withKeyDeserializerAndCoder( StringDeserializer.class, NullableCoder.of(StringUtf8Coder.of()).getValueCoder()) .withValueDeserializerAndCoder( - TaxiRidesKafkaAvroDeserializer.class, AvroCoder.of(TaxiRide.class)) + AvroDataClassKafkaAvroDeserializer.class, AvroCoder.of(AvroDataClass.class)) .withConsumerConfigUpdates(config) .withConsumerFactoryFn(new SslConsumerFactoryFn(sslConfig)) .withoutMetadata(); From 8a3d85d19e45613b28ad5bc0042921a0f961598c Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 23 Nov 2020 16:54:07 +0300 Subject: [PATCH 49/73] fix style --- .../java/org/apache/beam/templates/avro/AvroDataClass.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java index c3daedfa1ee5..9619c22f6c59 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java @@ -31,15 +31,10 @@ public class AvroDataClass { Float field2; Float field3; - public AvroDataClass( - String field1, - Float field2, - Float field3 - ) { + public AvroDataClass(String field1, Float field2, Float field3) { this.field1 = field1; this.field2 = field2; this.field3 = field3; - } public String getField1() { From 4172c96f2b7fe9119a5e05e38732c89fe5e868ff Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Fri, 27 Nov 2020 02:49:02 +0300 Subject: [PATCH 50/73] fixed review conversation items --- .../java/{ => kafka-to-pubsub}/README.md | 19 +++++++++++++------ .../apache/beam/templates/KafkaToPubsub.java | 2 +- 2 files changed, 14 insertions(+), 7 deletions(-) rename examples/templates/java/{ => kafka-to-pubsub}/README.md (87%) diff --git a/examples/templates/java/README.md b/examples/templates/java/kafka-to-pubsub/README.md similarity index 87% rename from examples/templates/java/README.md rename to examples/templates/java/kafka-to-pubsub/README.md index 097a6c41902e..4b3497008c7a 100644 --- a/examples/templates/java/README.md +++ b/examples/templates/java/kafka-to-pubsub/README.md @@ -157,7 +157,7 @@ on your project's [Container Registry](https://cloud.google.com/container-regist To execute the template you need to create the template spec file containing all the necessary information to run the job. This template already has the following -[metadata file](kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json) in resources. +[metadata file](src/main/resources/kafka_to_pubsub_metadata.json) in resources. Navigate to the template folder: @@ -176,6 +176,7 @@ gcloud dataflow flex-template build ${TEMPLATE_PATH} \ --jar "build/libs/beam-examples-templates-java-kafka-to-pubsub--all.jar" \ --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" ``` +Visit to Dataflow [documentation](https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates) page to get more information ### Create Dataflow Job Using the Apache Kafka to Google Pub/Sub Dataflow Flex Template @@ -228,13 +229,17 @@ You can do this in 3 different ways: ``` ## AVRO format transferring. -This template contains an example Class to deserialize AVRO from Kafka and serialize it to AVRO in Pub/Sub. + +This template contains an example demonstrating AVRO format support: +- Define custom Class to deserialize AVRO from Kafka [provided in example] +- Create custom data serialization in Apache Beam +- Serialize data to AVRO in Pub/Sub [provided in example]. To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [AvroDataClass](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/templates/avro/AvroDataClass.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java return KafkaIO.read() ... @@ -242,7 +247,9 @@ return KafkaIO.read() AvroDataClassKafkaAvroDeserializer.class, AvroCoder.of(AvroDataClass.class)) // put your classes here ... ``` -- Modify the write step in the [KafkaToPubsub class](kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. +- [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. +- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. + - NOTE: if it changed during the transform, you suold use changed one class definition. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { ... diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 33a3b54bedd5..a8beb0e4ead1 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -214,7 +214,7 @@ public static PipelineResult run(KafkaToPubsubOptions options) { .apply("createValues", Values.create()) .apply("writeAvrosToPubSub", PubsubIO.writeAvros(AvroDataClass.class)); - } else if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { + } else { pipeline .apply( "readFromKafka", From 3c63298179b3f7a485885e2de59994f6cf05897d Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 30 Nov 2020 17:02:38 +0300 Subject: [PATCH 51/73] fix getting ssl credentials from Vault --- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index a8beb0e4ead1..0103cff053d0 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -156,22 +156,23 @@ public static void main(String[] args) { public static PipelineResult run(KafkaToPubsubOptions options) { // Configure Kafka consumer properties Map kafkaConfig = new HashMap<>(); + Map sslConfig = new HashMap<>(); if (options.getSecretStoreUrl() != null && options.getVaultToken() != null) { Map> credentials = getKafkaCredentialsFromVault(options.getSecretStoreUrl(), options.getVaultToken()); kafkaConfig = configureKafka(credentials.get(KafkaPubsubConstants.KAFKA_CREDENTIALS)); + sslConfig = credentials.get(KafkaPubsubConstants.SSL_CREDENTIALS); } else { LOG.warn( "No information to retrieve Kafka credentials was provided. " + "Trying to initiate an unauthorized connection."); } - Map sslConfig = new HashMap<>(); if (isSslSpecified(options)) { sslConfig.putAll(configureSsl(options)); } else { LOG.info( - "No information to retrieve SSL certificate was provided. " + "No information to retrieve SSL certificate was provided by parameters." + "Trying to initiate a plain text connection."); } From 477df3e234f351e43f91105eef79ecf68eebbf56 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 30 Nov 2020 17:10:19 +0300 Subject: [PATCH 52/73] FIX add empty && null map validation to sslConfig --- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 0103cff053d0..0edb6d160db4 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -168,7 +168,11 @@ public static PipelineResult run(KafkaToPubsubOptions options) { + "Trying to initiate an unauthorized connection."); } - if (isSslSpecified(options)) { + if (sslConfig == null) { + sslConfig = new HashMap<>(); + } + + if (isSslSpecified(options) && sslConfig.isEmpty()) { sslConfig.putAll(configureSsl(options)); } else { LOG.info( From 83ff7b78aa84a9f7b79cb47ff896b10a41016d37 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 30 Nov 2020 18:06:02 +0300 Subject: [PATCH 53/73] FIX. remove vault ssl certs parameters --- .../main/java/org/apache/beam/templates/KafkaToPubsub.java | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java index 0edb6d160db4..68afee96feb8 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java @@ -161,18 +161,13 @@ public static PipelineResult run(KafkaToPubsubOptions options) { Map> credentials = getKafkaCredentialsFromVault(options.getSecretStoreUrl(), options.getVaultToken()); kafkaConfig = configureKafka(credentials.get(KafkaPubsubConstants.KAFKA_CREDENTIALS)); - sslConfig = credentials.get(KafkaPubsubConstants.SSL_CREDENTIALS); } else { LOG.warn( "No information to retrieve Kafka credentials was provided. " + "Trying to initiate an unauthorized connection."); } - if (sslConfig == null) { - sslConfig = new HashMap<>(); - } - - if (isSslSpecified(options) && sslConfig.isEmpty()) { + if (isSslSpecified(options)) { sslConfig.putAll(configureSsl(options)); } else { LOG.info( From b1ba8e123a2000ff14a6dd7b4118cd296654df3b Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 30 Nov 2020 18:27:05 +0300 Subject: [PATCH 54/73] metadata fix --- .../src/main/resources/kafka_to_pubsub_metadata.json | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json index 65469c4bf78f..567ebca02764 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ b/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json @@ -79,6 +79,14 @@ "regexes": [], "paramType": "TEXT", "isOptional": true + }, + { + "name": "keyPassword", + "label": "Key Password", + "helpText": "The store password for the key store password", + "regexes": [], + "paramType": "TEXT", + "isOptional": true } ] } From 08bc3fefca612d43d1abab890c7d96fc8aa8bf15 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Mon, 30 Nov 2020 18:38:37 +0300 Subject: [PATCH 55/73] Local paths fix for SSL from GCS --- .../beam/templates/kafka/consumer/SslConsumerFactoryFn.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java index 6b6b29e0088f..eac7bd550f8c 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java @@ -57,6 +57,7 @@ public Consumer apply(Map config) { String truststoreLocation = sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG); if (truststoreLocation.startsWith("gs://")) { getGcsFileAsLocal(truststoreLocation, TRUSTSTORE_LOCAL_PATH); + sslConfig.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, TRUSTSTORE_LOCAL_PATH); } else { checkFileExists(truststoreLocation); } @@ -64,6 +65,7 @@ public Consumer apply(Map config) { String keystoreLocation = sslConfig.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); if (keystoreLocation.startsWith("gs://")) { getGcsFileAsLocal(keystoreLocation, KEYSTORE_LOCAL_PATH); + sslConfig.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, KEYSTORE_LOCAL_PATH); } else { checkFileExists(keystoreLocation); } From 9fb43b63430253f4a2b3cd57de7b9886ab4b3f67 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 1 Dec 2020 18:43:49 +0300 Subject: [PATCH 56/73] add new log message to avoid wrong local files usage --- .../beam/templates/kafka/consumer/SslConsumerFactoryFn.java | 1 + 1 file changed, 1 insertion(+) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java index eac7bd550f8c..d98eba67252b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java @@ -94,6 +94,7 @@ public Consumer apply(Map config) { } private void checkFileExists(String filePath) throws IOException { + LOG.info("Trying to get file: {} locally. Local files don't support when in using distribute runner", filePath); File f = new File(filePath); if (f.exists()) { LOG.debug("{} exists", f.getAbsolutePath()); From e99a7d31210f202f80fab645ce4f787c849160f9 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 1 Dec 2020 18:48:59 +0300 Subject: [PATCH 57/73] fix style --- .../beam/templates/kafka/consumer/SslConsumerFactoryFn.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java index d98eba67252b..da6c7c737303 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java @@ -94,7 +94,9 @@ public Consumer apply(Map config) { } private void checkFileExists(String filePath) throws IOException { - LOG.info("Trying to get file: {} locally. Local files don't support when in using distribute runner", filePath); + LOG.info( + "Trying to get file: {} locally. Local files don't support when in using distribute runner", + filePath); File f = new File(filePath); if (f.exists()) { LOG.debug("{} exists", f.getAbsolutePath()); From 2ae7d024a5576823415a5448a29bff62c9b19893 Mon Sep 17 00:00:00 2001 From: ramazan-yapparov <75415515+ramazan-yapparov@users.noreply.github.com> Date: Fri, 4 Dec 2020 19:16:12 +0300 Subject: [PATCH 58/73] Moved kafka-to-pubsub to examples/ directory and updated README.md (#6) --- .../java => }/kafka-to-pubsub/README.md | 120 ++---------------- .../java => }/kafka-to-pubsub/build.gradle | 2 +- examples/kafka-to-pubsub/run.sh | 10 ++ .../beam/examples}/KafkaPubsubConstants.java | 2 +- .../apache/beam/examples}/KafkaToPubsub.java | 18 +-- .../beam/examples}/avro/AvroDataClass.java | 2 +- .../AvroDataClassKafkaAvroDeserializer.java | 2 +- .../beam/examples}/avro/package-info.java | 2 +- .../kafka/consumer/SslConsumerFactoryFn.java | 2 +- .../beam/examples}/kafka/consumer/Utils.java | 10 +- .../examples/kafka/consumer/package-info.java | 20 +++ .../options/KafkaToPubsubOptions.java | 4 +- .../beam/examples}/options/package-info.java | 2 +- .../apache/beam/examples}/package-info.java | 2 +- .../examples}/transforms/FormatTransform.java | 10 +- .../examples}/transforms/package-info.java | 2 +- .../resources/kafka_to_pubsub_metadata.json | 0 .../beam/examples}/KafkaToPubsubTest.java | 10 +- .../kafka/consumer/package-info.java | 20 --- settings.gradle | 4 +- 20 files changed, 75 insertions(+), 169 deletions(-) rename examples/{templates/java => }/kafka-to-pubsub/README.md (58%) rename examples/{templates/java => }/kafka-to-pubsub/build.gradle (95%) create mode 100755 examples/kafka-to-pubsub/run.sh rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/KafkaPubsubConstants.java (97%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/KafkaToPubsub.java (93%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/avro/AvroDataClass.java (97%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/avro/AvroDataClassKafkaAvroDeserializer.java (97%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/avro/package-info.java (95%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/kafka/consumer/SslConsumerFactoryFn.java (99%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/kafka/consumer/Utils.java (94%) create mode 100644 examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/options/KafkaToPubsubOptions.java (96%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/options/package-info.java (95%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/package-info.java (96%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/transforms/FormatTransform.java (94%) rename examples/{templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates => kafka-to-pubsub/src/main/java/org/apache/beam/examples}/transforms/package-info.java (94%) rename examples/{templates/java => }/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json (100%) rename examples/{templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates => kafka-to-pubsub/src/test/java/org/apache/beam/examples}/KafkaToPubsubTest.java (90%) delete mode 100644 examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java diff --git a/examples/templates/java/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md similarity index 58% rename from examples/templates/java/kafka-to-pubsub/README.md rename to examples/kafka-to-pubsub/README.md index 4b3497008c7a..6fb46f065c19 100644 --- a/examples/templates/java/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -73,7 +73,7 @@ In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/john is used. It creates the `shadowJar` task that builds the Uber JAR: ``` -./gradlew -p examples/templates/java/kafka-to-pubsub clean shadowJar +./gradlew -p examples/kafka-to-pubsub clean shadowJar ``` ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains @@ -87,11 +87,14 @@ To execute this pipeline locally, specify the parameters: - Kafka Bootstrap servers - Kafka input topics - Pub/Sub output topic +- Output format + in the following format: ```bash --bootstrapServers=host:port \ --inputTopics=your-input-topic \ ---outputTopic=projects/your-project-id/topics/your-topic-pame +--outputTopic=projects/your-project-id/topics/your-topic-pame \ +--outputFormat=AVRO|PUBSUB ``` Optionally, to retrieve Kafka credentials for SASL/SCRAM, specify a URL to the credentials in HashiCorp Vault and the vault access token: @@ -119,114 +122,9 @@ To change the runner, specify: ``` See examples/java/README.md for steps and examples to configure different runners. -## Google Dataflow Template - -### Setting Up Project Environment - -#### Pipeline variables: - -``` -PROJECT=id-of-my-project -BUCKET_NAME=my-bucket -REGION=my-region -``` - -#### Template Metadata Storage Bucket Creation - -The Dataflow Flex template has to store its metadata in a bucket in -[Google Cloud Storage](https://cloud.google.com/storage), so it can be executed from the Google Cloud Platform. -Create the bucket in Google Cloud Storage if it doesn't exist yet: - -``` -gsutil mb gs://${BUCKET_NAME} -``` - -#### Containerization variables: - -``` -IMAGE_NAME=my-image-name -TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME} -BASE_CONTAINER_IMAGE=my-base-container-image -TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json" -``` - -### Creating the Dataflow Flex Template - -Dataflow Flex Templates package the pipeline as a Docker image and stage these images -on your project's [Container Registry](https://cloud.google.com/container-registry). - -To execute the template you need to create the template spec file containing all -the necessary information to run the job. This template already has the following -[metadata file](src/main/resources/kafka_to_pubsub_metadata.json) in resources. - -Navigate to the template folder: - -``` -cd /path/to/beam/examples/templates/java/kafka-to-pubsub -``` - -Build the Dataflow Flex Template: - -``` -gcloud dataflow flex-template build ${TEMPLATE_PATH} \ - --image-gcr-path ${TARGET_GCR_IMAGE} \ - --sdk-language "JAVA" \ - --flex-template-base-image ${BASE_CONTAINER_IMAGE} \ - --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \ - --jar "build/libs/beam-examples-templates-java-kafka-to-pubsub--all.jar" \ - --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub" -``` -Visit to Dataflow [documentation](https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates) page to get more information - -### Create Dataflow Job Using the Apache Kafka to Google Pub/Sub Dataflow Flex Template - -To deploy the pipeline, you should refer to the template file and pass the -[parameters](https://cloud.google.com/dataflow/docs/guides/specifying-exec-params#setting-other-cloud-dataflow-pipeline-options) -required by the pipeline. - -To understand how works flex templates, please take a look at this [Documentation](https://cloud.google.com/dataflow/docs/guides/templates/using-flex-templates) - -You can do this in 3 different ways: -1. Using [Dataflow Google Cloud Console](https://console.cloud.google.com/dataflow/jobs) - -2. Using `gcloud` CLI tool - ``` - gcloud dataflow flex-template run "kafka-to-pubsub-`date +%Y%m%d-%H%M%S`" \ - --template-file-gcs-location "${TEMPLATE_PATH}" \ - --parameters bootstrapServers="broker_1:9092,broker_2:9092" \ - --parameters inputTopics="topic1,topic2" \ - --parameters outputTopic="projects/${PROJECT}/topics/your-topic-name" \ - --parameters outputFormat="PLAINTEXT" \ - --parameters secretStoreUrl="http(s)://host:port/path/to/credentials" \ - --parameters vaultToken="your-token" \ - --region "${REGION}" - ``` -3. With a REST API request - ``` - API_ROOT_URL="https://dataflow.googleapis.com" - TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/locations/${REGION}/flexTemplates:launch" - JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`" +## Google Dataflow Execution - time curl -X POST -H "Content-Type: application/json" \ - -H "Authorization: Bearer $(gcloud auth print-access-token)" \ - -d ' - { - "launch_parameter": { - "jobName": "'$JOB_NAME'", - "containerSpecGcsPath": "'$TEMPLATE_PATH'", - "parameters": { - "bootstrapServers": "broker_1:9091, broker_2:9092", - "inputTopics": "topic1, topic2", - "outputTopic": "projects/'$PROJECT'/topics/your-topic-name", - "outputFormat": "PLAINTEXT", - "secretStoreUrl": "http(s)://host:port/path/to/credentials", - "vaultToken": "your-token" - } - } - } - ' - "${TEMPLATES_LAUNCH_API}" - ``` +This example also exists as Google Dataflow Template, see its [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/kafka-to-pubsub/README.md) for more information. ## AVRO format transferring. @@ -249,7 +147,7 @@ return KafkaIO.read() ``` - [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. - Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. - - NOTE: if it changed during the transform, you suold use changed one class definition. + - NOTE: if it changed during the transform, you should use changed one class definition. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { ... @@ -258,4 +156,4 @@ if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { } ``` -_Note: The Kafka to Pub/Sub Dataflow Flex template supports SSL configuration with the certificate located only in GCS._ +_Note: The Kafka to Pub/Sub job executed with a distributed runner supports SSL configuration with the certificate located only in GCS._ diff --git a/examples/templates/java/kafka-to-pubsub/build.gradle b/examples/kafka-to-pubsub/build.gradle similarity index 95% rename from examples/templates/java/kafka-to-pubsub/build.gradle rename to examples/kafka-to-pubsub/build.gradle index d311b10c0251..b4db644e0a26 100644 --- a/examples/templates/java/kafka-to-pubsub/build.gradle +++ b/examples/kafka-to-pubsub/build.gradle @@ -23,7 +23,7 @@ plugins { } applyJavaNature( exportJavadoc: false, - automaticModuleName: 'org.apache.beam.examples.templates.kafka-to-pubsub') + automaticModuleName: 'org.apache.beam.examples.kafka-to-pubsub') provideIntegrationTestingDependencies() enableJavaPerformanceTesting() diff --git a/examples/kafka-to-pubsub/run.sh b/examples/kafka-to-pubsub/run.sh new file mode 100755 index 000000000000..38b063a4734a --- /dev/null +++ b/examples/kafka-to-pubsub/run.sh @@ -0,0 +1,10 @@ +#!/usr/bin/env bash + +java -cp build/libs/beam-examples-kafka-to-pubsub-2.27.0-SNAPSHOT.jar org.apache.beam.examples.KafkaToPubsub \ + --runner=DataflowRunner \ + --project=ktp-295315 \ + --region=us-central1 \ + --bootstrapServers=localhost:9092 \ + --inputTopics=input \ + --outputTopic=projects/ktp-295315/topics/ktp-topic \ + --outputFormat=PUBSUB diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java similarity index 97% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java index 07c55d8c7746..1031a2ae888e 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaPubsubConstants.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates; +package org.apache.beam.examples; /** Constant variables that are used across the template's parts. */ public class KafkaPubsubConstants { diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java similarity index 93% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java index 68afee96feb8..b5d2df8bca2b 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/KafkaToPubsub.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates; +package org.apache.beam.examples; -import static org.apache.beam.templates.kafka.consumer.Utils.configureKafka; -import static org.apache.beam.templates.kafka.consumer.Utils.configureSsl; -import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; -import static org.apache.beam.templates.kafka.consumer.Utils.isSslSpecified; +import static org.apache.beam.examples.kafka.consumer.Utils.configureKafka; +import static org.apache.beam.examples.kafka.consumer.Utils.configureSsl; +import static org.apache.beam.examples.kafka.consumer.Utils.getKafkaCredentialsFromVault; +import static org.apache.beam.examples.kafka.consumer.Utils.isSslSpecified; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayList; @@ -33,10 +33,10 @@ import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.templates.avro.AvroDataClass; -import org.apache.beam.templates.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.templates.options.KafkaToPubsubOptions; -import org.apache.beam.templates.transforms.FormatTransform; +import org.apache.beam.examples.avro.AvroDataClass; +import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.options.KafkaToPubsubOptions; +import org.apache.beam.examples.transforms.FormatTransform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java similarity index 97% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java index 9619c22f6c59..5fbf63d3e9eb 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClass.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.avro; +package org.apache.beam.examples.avro; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java similarity index 97% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java index 119a2162aaa8..837873821451 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.avro; +package org.apache.beam.examples.avro; import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java similarity index 95% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java index 0a3040b226eb..fa65d14948d4 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/avro/package-info.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.templates.avro; +package org.apache.beam.examples.avro; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java similarity index 99% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java index da6c7c737303..6bba7ffc1545 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.kafka.consumer; +package org.apache.beam.examples.kafka.consumer; import java.io.File; import java.io.IOException; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java similarity index 94% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java index 5b87750651c7..9e83cbf3026e 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/Utils.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.kafka.consumer; +package org.apache.beam.examples.kafka.consumer; -import static org.apache.beam.templates.KafkaPubsubConstants.KAFKA_CREDENTIALS; -import static org.apache.beam.templates.KafkaPubsubConstants.PASSWORD; -import static org.apache.beam.templates.KafkaPubsubConstants.USERNAME; +import static org.apache.beam.examples.KafkaPubsubConstants.KAFKA_CREDENTIALS; +import static org.apache.beam.examples.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.examples.KafkaPubsubConstants.USERNAME; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.beam.templates.options.KafkaToPubsubOptions; +import org.apache.beam.examples.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.http.HttpResponse; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java new file mode 100644 index 000000000000..f83ad19d72a3 --- /dev/null +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template. */ +package org.apache.beam.examples.kafka.consumer; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java similarity index 96% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java index 92b24636dcc7..00e09f6294b6 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/KafkaToPubsubOptions.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.options; +package org.apache.beam.examples.options; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; -import org.apache.beam.templates.transforms.FormatTransform; +import org.apache.beam.examples.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { @Description( diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java similarity index 95% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java index eb553043ce1f..cc5edf355513 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/options/package-info.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.templates.options; +package org.apache.beam.examples.options; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java similarity index 96% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java index 0a83c30478ce..9de2d2e771d9 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/package-info.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.templates; +package org.apache.beam.examples; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java similarity index 94% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java index fbb67044ddf9..fd2e022118f3 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/FormatTransform.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates.transforms; +package org.apache.beam.examples.transforms; import java.util.List; import java.util.Map; @@ -32,10 +32,10 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.templates.avro.AvroDataClass; -import org.apache.beam.templates.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.templates.kafka.consumer.SslConsumerFactoryFn; -import org.apache.beam.templates.options.KafkaToPubsubOptions; +import org.apache.beam.examples.avro.AvroDataClass; +import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.kafka.consumer.SslConsumerFactoryFn; +import org.apache.beam.examples.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.kafka.common.serialization.StringDeserializer; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java similarity index 94% rename from examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java rename to examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java index 3ca0b000a644..2a0f2eabf3af 100644 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/transforms/package-info.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.templates.transforms; +package org.apache.beam.examples.transforms; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json similarity index 100% rename from examples/templates/java/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json rename to examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json diff --git a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java b/examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java similarity index 90% rename from examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java rename to examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java index 81aa9f0bf534..063d09f09d9a 100644 --- a/examples/templates/java/kafka-to-pubsub/src/test/java/org/apache/beam/templates/KafkaToPubsubTest.java +++ b/examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java @@ -15,15 +15,15 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.templates; +package org.apache.beam.examples; -import static org.apache.beam.templates.KafkaPubsubConstants.PASSWORD; -import static org.apache.beam.templates.KafkaPubsubConstants.USERNAME; -import static org.apache.beam.templates.kafka.consumer.Utils.getKafkaCredentialsFromVault; +import static org.apache.beam.examples.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.examples.KafkaPubsubConstants.USERNAME; +import static org.apache.beam.examples.kafka.consumer.Utils.getKafkaCredentialsFromVault; import java.util.HashMap; import java.util.Map; -import org.apache.beam.templates.kafka.consumer.Utils; +import org.apache.beam.examples.kafka.consumer.Utils; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.security.scram.ScramMechanism; import org.junit.Assert; diff --git a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java b/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java deleted file mode 100644 index 43df3c742fd1..000000000000 --- a/examples/templates/java/kafka-to-pubsub/src/main/java/org/apache/beam/templates/kafka/consumer/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ - -/** Kafka to Pubsub template. */ -package org.apache.beam.templates.kafka.consumer; diff --git a/settings.gradle b/settings.gradle index 59a55e2aab14..7092d9f33f08 100644 --- a/settings.gradle +++ b/settings.gradle @@ -52,6 +52,7 @@ include ":release:go-licenses:py" include ":examples:java" include ":examples:kotlin" +include ":examples:kafka-to-pubsub" include ":model:fn-execution" include ":model:job-management" include ":model:pipeline" @@ -216,6 +217,3 @@ include "beam-test-tools" project(":beam-test-tools").dir = file(".test-infra/tools") include "beam-test-jenkins" project(":beam-test-jenkins").dir = file(".test-infra/jenkins") -include ':examples:templates:java' -include ':examples:templates:java:kafka-to-pubsub' - From 3e558a39aeed7671d4d9ae1f630d22f41e354044 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Fri, 4 Dec 2020 21:07:01 +0300 Subject: [PATCH 59/73] Stylefix --- .../main/java/org/apache/beam/examples/KafkaToPubsub.java | 8 ++++---- .../beam/examples/options/KafkaToPubsubOptions.java | 2 +- .../apache/beam/examples/transforms/FormatTransform.java | 8 ++++---- 3 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java index b5d2df8bca2b..e2de55f11916 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java @@ -28,15 +28,15 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import org.apache.beam.examples.avro.AvroDataClass; +import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.options.KafkaToPubsubOptions; +import org.apache.beam.examples.transforms.FormatTransform; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.transforms.Values; -import org.apache.beam.examples.avro.AvroDataClass; -import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.examples.options.KafkaToPubsubOptions; -import org.apache.beam.examples.transforms.FormatTransform; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java index 00e09f6294b6..7aa1950276a4 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java @@ -17,10 +17,10 @@ */ package org.apache.beam.examples.options; +import org.apache.beam.examples.transforms.FormatTransform; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; -import org.apache.beam.examples.transforms.FormatTransform; public interface KafkaToPubsubOptions extends PipelineOptions { @Description( diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java index fd2e022118f3..bf3f2b91512e 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java @@ -19,6 +19,10 @@ import java.util.List; import java.util.Map; +import org.apache.beam.examples.avro.AvroDataClass; +import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.kafka.consumer.SslConsumerFactoryFn; +import org.apache.beam.examples.options.KafkaToPubsubOptions; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; @@ -32,10 +36,6 @@ import org.apache.beam.sdk.values.PCollection; import org.apache.beam.sdk.values.PDone; import org.apache.beam.sdk.values.TypeDescriptor; -import org.apache.beam.examples.avro.AvroDataClass; -import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.examples.kafka.consumer.SslConsumerFactoryFn; -import org.apache.beam.examples.options.KafkaToPubsubOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Charsets; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; import org.apache.kafka.common.serialization.StringDeserializer; From 51ddeadef3daf09f1d2af569d5d747c185a5f529 Mon Sep 17 00:00:00 2001 From: Ramazan Yapparov Date: Mon, 7 Dec 2020 14:46:30 +0300 Subject: [PATCH 60/73] Removed unused file --- examples/kafka-to-pubsub/run.sh | 10 ---------- 1 file changed, 10 deletions(-) delete mode 100755 examples/kafka-to-pubsub/run.sh diff --git a/examples/kafka-to-pubsub/run.sh b/examples/kafka-to-pubsub/run.sh deleted file mode 100755 index 38b063a4734a..000000000000 --- a/examples/kafka-to-pubsub/run.sh +++ /dev/null @@ -1,10 +0,0 @@ -#!/usr/bin/env bash - -java -cp build/libs/beam-examples-kafka-to-pubsub-2.27.0-SNAPSHOT.jar org.apache.beam.examples.KafkaToPubsub \ - --runner=DataflowRunner \ - --project=ktp-295315 \ - --region=us-central1 \ - --bootstrapServers=localhost:9092 \ - --inputTopics=input \ - --outputTopic=projects/ktp-295315/topics/ktp-topic \ - --outputFormat=PUBSUB From e8ca92d68646cec459d3ecd94b0efe7551699268 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 7 Dec 2020 23:12:26 +0300 Subject: [PATCH 61/73] add tbd section for e-2-e tests --- examples/kafka-to-pubsub/README.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/examples/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md index 6fb46f065c19..f6f9ee271da3 100644 --- a/examples/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -57,6 +57,7 @@ This section describes what is needed to get the template up and running. - Creating the Dataflow Flex Template - Create a Dataflow job to ingest data using the template. - Avro format transferring. +- E2E tests (TBD) ## Assembling the Uber-JAR @@ -156,4 +157,7 @@ if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { } ``` +## End to end tests +TBD + _Note: The Kafka to Pub/Sub job executed with a distributed runner supports SSL configuration with the certificate located only in GCS._ From 880f7e6d00ad6e582a0edce14c4bdaff4173313e Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Mon, 7 Dec 2020 23:13:07 +0300 Subject: [PATCH 62/73] fix styles --- examples/kafka-to-pubsub/README.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/examples/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md index f6f9ee271da3..f678779fb6ed 100644 --- a/examples/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -55,7 +55,7 @@ This section describes what is needed to get the template up and running. - Google Dataflow Template - Set up the environment - Creating the Dataflow Flex Template - - Create a Dataflow job to ingest data using the template. + - Create a Dataflow job to ingest data using the template - Avro format transferring. - E2E tests (TBD) @@ -157,7 +157,7 @@ if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { } ``` -## End to end tests +## End to end tests TBD _Note: The Kafka to Pub/Sub job executed with a distributed runner supports SSL configuration with the certificate located only in GCS._ From 0eba7ba5c7661c1e8a60178103f698064686eda4 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 8 Dec 2020 01:48:02 +0300 Subject: [PATCH 63/73] specifying kafka-clients version --- examples/kafka-to-pubsub/build.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/kafka-to-pubsub/build.gradle b/examples/kafka-to-pubsub/build.gradle index b4db644e0a26..ac09b16cb7b8 100644 --- a/examples/kafka-to-pubsub/build.gradle +++ b/examples/kafka-to-pubsub/build.gradle @@ -37,7 +37,7 @@ repositories { } dependencies { - compile library.java.kafka_clients + compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.6.0' compile project(path: ":sdks:java:core", configuration: "shadow") compile project(":sdks:java:extensions:google-cloud-platform-core") compile project(":sdks:java:io:google-cloud-platform") From 5919f9b15047c6308bac5d8b7134895023cc0a75 Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 8 Dec 2020 02:33:32 +0300 Subject: [PATCH 64/73] fix readme --- examples/kafka-to-pubsub/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md index f678779fb6ed..a3399c4c2445 100644 --- a/examples/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -19,7 +19,7 @@ # Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains an [Apache Beam](https://beam.apache.org/) Template that creates a pipeline +This directory contains an [Apache Beam](https://beam.apache.org/) pipeline example that creates a pipeline to read data from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Cloud Pub/Sub](https://cloud.google.com/pubsub). From 4372d6ff56bdef40e106f358adb1881cd3032e6f Mon Sep 17 00:00:00 2001 From: Ilya Kozyrev Date: Tue, 8 Dec 2020 02:39:57 +0300 Subject: [PATCH 65/73] template -> exmples --- examples/kafka-to-pubsub/README.md | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md index a3399c4c2445..f27dbc4f8af8 100644 --- a/examples/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -17,7 +17,7 @@ under the License. --> -# Apache Beam Template to ingest data from Apache Kafka to Google Cloud Pub/Sub +# Apache Beam pipeline example to ingest data from Apache Kafka to Google Cloud Pub/Sub This directory contains an [Apache Beam](https://beam.apache.org/) pipeline example that creates a pipeline to read data from a single or multiple topics from @@ -36,7 +36,7 @@ Supported input source configurations: Supported destination configuration: - Single Google Cloud Pub/Sub topic. -In a simple scenario, the template will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The template supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the template will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. +In a simple scenario, the example will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The example supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the example will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. ## Requirements @@ -49,7 +49,7 @@ In a simple scenario, the template will create an Apache Beam pipeline that will ## Getting Started -This section describes what is needed to get the template up and running. +This section describes what is needed to get the exaple up and running. - Assembling the Uber-JAR - Local execution - Google Dataflow Template @@ -61,7 +61,7 @@ This section describes what is needed to get the template up and running. ## Assembling the Uber-JAR -To run this template the template Java project should be built into +To run this example the Java project should be built into an Uber JAR file. Navigate to the Beam folder: @@ -129,16 +129,16 @@ This example also exists as Google Dataflow Template, see its [README.md](https: ## AVRO format transferring. -This template contains an example demonstrating AVRO format support: +This example contains an example demonstrating AVRO format support: - Define custom Class to deserialize AVRO from Kafka [provided in example] - Create custom data serialization in Apache Beam - Serialize data to AVRO in Pub/Sub [provided in example]. To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/templates/avro/AvroDataClass.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/templates/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/templates/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/examples/avro/AvroDataClass.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/examples/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java return KafkaIO.read() ... @@ -147,7 +147,7 @@ return KafkaIO.read() ... ``` - [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. -- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/templates/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. +- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/examples/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. - NOTE: if it changed during the transform, you should use changed one class definition. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { From dd85c93e8d405bb3a936ae5a14eca14825f89eaf Mon Sep 17 00:00:00 2001 From: Ilya Date: Tue, 8 Dec 2020 22:46:41 +0300 Subject: [PATCH 66/73] Update examples/kafka-to-pubsub/README.md Co-authored-by: Brian Hulette --- examples/kafka-to-pubsub/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/kafka-to-pubsub/README.md b/examples/kafka-to-pubsub/README.md index f27dbc4f8af8..11259d3cf9a3 100644 --- a/examples/kafka-to-pubsub/README.md +++ b/examples/kafka-to-pubsub/README.md @@ -117,7 +117,7 @@ specify the parameters: --keystorePassword=your-keystore-password --keyPassword=your-key-password ``` -To change the runner, specify: +By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: ```bash --runner=YOUR_SELECTED_RUNNER ``` From 51182eb24d4d7f2c7f424259f5287aed26dd67af Mon Sep 17 00:00:00 2001 From: Ramazan Yapparov Date: Wed, 9 Dec 2020 11:58:27 +0300 Subject: [PATCH 67/73] Fixed outdated import --- .../java/org/apache/beam/examples/kafka/consumer/Utils.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java index 9e83cbf3026e..13f395a4bf21 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java +++ b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java @@ -34,7 +34,7 @@ import org.apache.http.util.EntityUtils; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.config.SslConfigs; -import org.apache.kafka.common.security.scram.ScramMechanism; +import org.apache.kafka.common.security.scram.internals.ScramMechanism; import org.checkerframework.checker.nullness.qual.Nullable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; From a5406a558b095ea2eded4536d1bcb9aed1c8e449 Mon Sep 17 00:00:00 2001 From: Ramazan Yapparov Date: Wed, 9 Dec 2020 17:56:51 +0300 Subject: [PATCH 68/73] Moved template to examples/complete --- examples/java/build.gradle | 9 +- .../kafkatopubsub}/KafkaPubsubConstants.java | 2 +- .../kafkatopubsub}/KafkaToPubsub.java | 18 ++-- .../complete/kafkatopubsub}/README.md | 0 .../kafkatopubsub}/avro/AvroDataClass.java | 2 +- .../AvroDataClassKafkaAvroDeserializer.java | 2 +- .../kafkatopubsub}/avro/package-info.java | 2 +- .../kafka/consumer/SslConsumerFactoryFn.java | 2 +- .../kafkatopubsub}/kafka/consumer/Utils.java | 10 +- .../kafka/consumer/package-info.java | 20 ++++ .../options/KafkaToPubsubOptions.java | 4 +- .../kafkatopubsub/options}/package-info.java | 2 +- .../complete/kafkatopubsub}/package-info.java | 2 +- .../transforms/FormatTransform.java | 10 +- .../transforms}/package-info.java | 2 +- .../kafkatopubsub}/KafkaToPubsubTest.java | 12 +-- examples/kafka-to-pubsub/build.gradle | 49 ---------- .../examples/transforms/package-info.java | 20 ---- .../resources/kafka_to_pubsub_metadata.json | 92 ------------------- settings.gradle | 1 - 20 files changed, 63 insertions(+), 198 deletions(-) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/KafkaPubsubConstants.java (95%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/KafkaToPubsub.java (91%) rename examples/{kafka-to-pubsub => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/README.md (100%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/avro/AvroDataClass.java (96%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/avro/AvroDataClassKafkaAvroDeserializer.java (96%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/avro/package-info.java (93%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/kafka/consumer/SslConsumerFactoryFn.java (98%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/kafka/consumer/Utils.java (93%) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/package-info.java rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/options/KafkaToPubsubOptions.java (95%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options}/package-info.java (92%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/package-info.java (93%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub}/transforms/FormatTransform.java (92%) rename examples/{kafka-to-pubsub/src/main/java/org/apache/beam/examples/options => java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms}/package-info.java (92%) rename examples/{kafka-to-pubsub/src/test/java/org/apache/beam/examples => java/src/test/java/org/apache/beam/examples/complete/kafkatopubsub}/KafkaToPubsubTest.java (85%) delete mode 100644 examples/kafka-to-pubsub/build.gradle delete mode 100644 examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java delete mode 100644 examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json diff --git a/examples/java/build.gradle b/examples/java/build.gradle index c305f0451f09..a1b6827a98ec 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -18,7 +18,12 @@ import groovy.json.JsonOutput -plugins { id 'org.apache.beam.module' } +plugins { + id 'java' + id 'org.apache.beam.module' + id 'com.github.johnrengelman.shadow' +} + applyJavaNature( exportJavadoc: false, automaticModuleName: 'org.apache.beam.examples', @@ -49,9 +54,11 @@ configurations.sparkRunnerPreCommit { dependencies { compile enforcedPlatform(library.java.google_cloud_platform_libraries_bom) compile library.java.vendored_guava_26_0_jre + compile library.java.kafka_clients compile project(path: ":sdks:java:core", 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") compile project(":sdks:java:extensions:ml") compile library.java.avro compile library.java.bigdataoss_util diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaPubsubConstants.java similarity index 95% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaPubsubConstants.java index 1031a2ae888e..46f021a5ca75 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaPubsubConstants.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaPubsubConstants.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples; +package org.apache.beam.examples.complete.kafkatopubsub; /** Constant variables that are used across the template's parts. */ public class KafkaPubsubConstants { diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java similarity index 91% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java index e2de55f11916..a5155428292a 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/KafkaToPubsub.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java @@ -15,12 +15,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples; +package org.apache.beam.examples.complete.kafkatopubsub; -import static org.apache.beam.examples.kafka.consumer.Utils.configureKafka; -import static org.apache.beam.examples.kafka.consumer.Utils.configureSsl; -import static org.apache.beam.examples.kafka.consumer.Utils.getKafkaCredentialsFromVault; -import static org.apache.beam.examples.kafka.consumer.Utils.isSslSpecified; +import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.configureKafka; +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.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; import java.util.ArrayList; @@ -28,10 +28,10 @@ import java.util.HashMap; import java.util.List; import java.util.Map; -import org.apache.beam.examples.avro.AvroDataClass; -import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.examples.options.KafkaToPubsubOptions; -import org.apache.beam.examples.transforms.FormatTransform; +import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClass; +import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions; +import org.apache.beam.examples.complete.kafkatopubsub.transforms.FormatTransform; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; import org.apache.beam.sdk.io.gcp.pubsub.PubsubIO; diff --git a/examples/kafka-to-pubsub/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md similarity index 100% rename from examples/kafka-to-pubsub/README.md rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java similarity index 96% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java index 5fbf63d3e9eb..8c8702115f65 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClass.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.avro; +package org.apache.beam.examples.complete.kafkatopubsub.avro; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.DefaultCoder; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClassKafkaAvroDeserializer.java similarity index 96% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClassKafkaAvroDeserializer.java index 837873821451..a9aeb72eb196 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClassKafkaAvroDeserializer.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.avro; +package org.apache.beam.examples.complete.kafkatopubsub.avro; import io.confluent.kafka.serializers.AbstractKafkaAvroDeserializer; import io.confluent.kafka.serializers.KafkaAvroDeserializerConfig; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/package-info.java similarity index 93% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/package-info.java index fa65d14948d4..1dc4f36ce656 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/avro/package-info.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.examples.avro; +package org.apache.beam.examples.complete.kafkatopubsub.avro; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/SslConsumerFactoryFn.java similarity index 98% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/SslConsumerFactoryFn.java index 6bba7ffc1545..012f147b77fd 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/SslConsumerFactoryFn.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/SslConsumerFactoryFn.java @@ -15,7 +15,7 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.kafka.consumer; +package org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer; import java.io.File; import java.io.IOException; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java similarity index 93% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java index 13f395a4bf21..ff4cf5f3dcd8 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/Utils.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/Utils.java @@ -15,16 +15,16 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.kafka.consumer; +package org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer; -import static org.apache.beam.examples.KafkaPubsubConstants.KAFKA_CREDENTIALS; -import static org.apache.beam.examples.KafkaPubsubConstants.PASSWORD; -import static org.apache.beam.examples.KafkaPubsubConstants.USERNAME; +import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.KAFKA_CREDENTIALS; +import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.USERNAME; import java.io.IOException; import java.util.HashMap; import java.util.Map; -import org.apache.beam.examples.options.KafkaToPubsubOptions; +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.http.HttpResponse; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/package-info.java new file mode 100644 index 000000000000..54aff5e766e7 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/kafka/consumer/package-info.java @@ -0,0 +1,20 @@ +/* + * 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. + */ + +/** Kafka to Pubsub template. */ +package org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/KafkaToPubsubOptions.java similarity index 95% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/KafkaToPubsubOptions.java index 7aa1950276a4..7e0460571ef1 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/KafkaToPubsubOptions.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/KafkaToPubsubOptions.java @@ -15,9 +15,9 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.options; +package org.apache.beam.examples.complete.kafkatopubsub.options; -import org.apache.beam.examples.transforms.FormatTransform; +import org.apache.beam.examples.complete.kafkatopubsub.transforms.FormatTransform; import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/package-info.java similarity index 92% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/package-info.java index f83ad19d72a3..361a5eab058c 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/kafka/consumer/package-info.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/options/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.examples.kafka.consumer; +package org.apache.beam.examples.complete.kafkatopubsub.options; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/package-info.java similarity index 93% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/package-info.java index 9de2d2e771d9..66b4c04f763f 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/package-info.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.examples; +package org.apache.beam.examples.complete.kafkatopubsub; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java similarity index 92% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java index bf3f2b91512e..b6c5846f83ce 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/FormatTransform.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java @@ -15,14 +15,14 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples.transforms; +package org.apache.beam.examples.complete.kafkatopubsub.transforms; import java.util.List; import java.util.Map; -import org.apache.beam.examples.avro.AvroDataClass; -import org.apache.beam.examples.avro.AvroDataClassKafkaAvroDeserializer; -import org.apache.beam.examples.kafka.consumer.SslConsumerFactoryFn; -import org.apache.beam.examples.options.KafkaToPubsubOptions; +import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClass; +import org.apache.beam.examples.complete.kafkatopubsub.avro.AvroDataClassKafkaAvroDeserializer; +import org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.SslConsumerFactoryFn; +import org.apache.beam.examples.complete.kafkatopubsub.options.KafkaToPubsubOptions; import org.apache.beam.sdk.coders.AvroCoder; import org.apache.beam.sdk.coders.NullableCoder; import org.apache.beam.sdk.coders.StringUtf8Coder; diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/package-info.java similarity index 92% rename from examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java rename to examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/package-info.java index cc5edf355513..83b9d5c3c956 100644 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/options/package-info.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/package-info.java @@ -17,4 +17,4 @@ */ /** Kafka to Pubsub template. */ -package org.apache.beam.examples.options; +package org.apache.beam.examples.complete.kafkatopubsub.transforms; diff --git a/examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java b/examples/java/src/test/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsubTest.java similarity index 85% rename from examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java rename to examples/java/src/test/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsubTest.java index 063d09f09d9a..e71b30459723 100644 --- a/examples/kafka-to-pubsub/src/test/java/org/apache/beam/examples/KafkaToPubsubTest.java +++ b/examples/java/src/test/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsubTest.java @@ -15,17 +15,17 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.beam.examples; +package org.apache.beam.examples.complete.kafkatopubsub; -import static org.apache.beam.examples.KafkaPubsubConstants.PASSWORD; -import static org.apache.beam.examples.KafkaPubsubConstants.USERNAME; -import static org.apache.beam.examples.kafka.consumer.Utils.getKafkaCredentialsFromVault; +import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.PASSWORD; +import static org.apache.beam.examples.complete.kafkatopubsub.KafkaPubsubConstants.USERNAME; +import static org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils.getKafkaCredentialsFromVault; import java.util.HashMap; import java.util.Map; -import org.apache.beam.examples.kafka.consumer.Utils; +import org.apache.beam.examples.complete.kafkatopubsub.kafka.consumer.Utils; import org.apache.kafka.common.config.SaslConfigs; -import org.apache.kafka.common.security.scram.ScramMechanism; +import org.apache.kafka.common.security.scram.internals.ScramMechanism; import org.junit.Assert; import org.junit.Test; import org.junit.runner.RunWith; diff --git a/examples/kafka-to-pubsub/build.gradle b/examples/kafka-to-pubsub/build.gradle deleted file mode 100644 index ac09b16cb7b8..000000000000 --- a/examples/kafka-to-pubsub/build.gradle +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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. - */ - -plugins { - id 'java' - id 'org.apache.beam.module' - id 'com.github.johnrengelman.shadow' -} -applyJavaNature( - exportJavadoc: false, - automaticModuleName: 'org.apache.beam.examples.kafka-to-pubsub') -provideIntegrationTestingDependencies() -enableJavaPerformanceTesting() - -description = "Apache Beam :: Templates :: Kafka to PubSub" -ext.summary = """Apache Beam SDK provides a simple, Java-based -interface for processing virtually any size data. This -artifact includes Kafka to PubSub template for Google DataFlow""" - -repositories { - mavenCentral() -} - -dependencies { - compile group: 'org.apache.kafka', name: 'kafka-clients', version: '2.6.0' - compile project(path: ":sdks:java:core", 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") - compile project(":runners:google-cloud-dataflow-java") - runtime project(path: ":runners:direct-java", configuration: "shadow") - testCompile group: 'junit', name: 'junit', version: '4.12' -} - diff --git a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java b/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java deleted file mode 100644 index 2a0f2eabf3af..000000000000 --- a/examples/kafka-to-pubsub/src/main/java/org/apache/beam/examples/transforms/package-info.java +++ /dev/null @@ -1,20 +0,0 @@ -/* - * 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. - */ - -/** Kafka to Pubsub template. */ -package org.apache.beam.examples.transforms; diff --git a/examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json b/examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json deleted file mode 100644 index 567ebca02764..000000000000 --- a/examples/kafka-to-pubsub/src/main/resources/kafka_to_pubsub_metadata.json +++ /dev/null @@ -1,92 +0,0 @@ -{ - "name": "Kafka to Pub/Sub", - "description": "Streaming pipeline. Ingests JSON-encoded messages from Kafka to a Pub/Sub subscription", - "parameters": [ - { - "name": "bootstrapServers", - "label": "Kafka Bootstrap Servers", - "helpText": "Comma separated Kafka bootstrap servers in format ip:port", - "regexes": [], - "paramType": "TEXT" - }, - { - "name": "inputTopics", - "label": "Kafka input topics", - "helpText": "Comma separated Kafka topics to read data", - "regexes": [], - "paramType": "TEXT" - }, - { - "name": "outputTopic", - "label": "Pub/Sub output topic", - "helpText": "Pub/Sub topic to write the output, in the format of 'projects/yourproject/topics/yourtopic'", - "regexes": [], - "paramType": "TEXT" - }, - { - "name": "outputFormat", - "label": "Message format that is written into Pub/Sub", - "helpText": "One of two message formats that is written to Pub/Sub. PUBSUB or PLAINTEXT. Default is PLAINTEXT", - "regexes": [ - "^(PUBSUB|AVRO)$" - ], - "isOptional": true - }, - { - "name": "secretStoreUrl", - "label": "URL to credentials in Vault", - "helpText": "URL to Kafka credentials in HashiCorp Vault secret storage in format 'http(s)://vaultip:vaultport/path/to/credentials'", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "vaultToken", - "label": "Vault token", - "helpText": "Token to access HashiCorp Vault secret storage", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "truststorePath", - "label": "Truststore Path", - "helpText": "The path to the trust store file", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "truststorePassword", - "label": "The password for the trust store password", - "helpText": "Token to access HashiCorp Vault secret storage", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "keystorePath", - "label": "The path to the key store file", - "helpText": "Token to access HashiCorp Vault secret storage", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "keystorePassword", - "label": "Keystore Password", - "helpText": "The store password for the key store password", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - }, - { - "name": "keyPassword", - "label": "Key Password", - "helpText": "The store password for the key store password", - "regexes": [], - "paramType": "TEXT", - "isOptional": true - } - ] -} diff --git a/settings.gradle b/settings.gradle index 7092d9f33f08..6ed2690706bc 100644 --- a/settings.gradle +++ b/settings.gradle @@ -52,7 +52,6 @@ include ":release:go-licenses:py" include ":examples:java" include ":examples:kotlin" -include ":examples:kafka-to-pubsub" include ":model:fn-execution" include ":model:job-management" include ":model:pipeline" From e7fc00e6df390e84d55cbc4e8e03c98b66f56625 Mon Sep 17 00:00:00 2001 From: Ramazan Yapparov Date: Wed, 9 Dec 2020 18:10:54 +0300 Subject: [PATCH 69/73] Updated paths in readme file --- .../beam/examples/complete/kafkatopubsub/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md index 11259d3cf9a3..283ca9dd281f 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md @@ -74,14 +74,14 @@ In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/john is used. It creates the `shadowJar` task that builds the Uber JAR: ``` -./gradlew -p examples/kafka-to-pubsub clean shadowJar +./gradlew -p examples/java clean shadowJar ``` ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains both target package *and* all its dependencies. The result of the `shadowJar` task execution is a `.jar` file that is generated -under the `build/libs/` folder in kafka-to-pubsub directory. +under the `build/libs/` folder in examples/java directory. ## Local execution To execute this pipeline locally, specify the parameters: @@ -136,9 +136,9 @@ This example contains an example demonstrating AVRO format support: To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/examples/avro/AvroDataClass.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/examples/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/examples/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java return KafkaIO.read() ... @@ -147,7 +147,7 @@ return KafkaIO.read() ... ``` - [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. -- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/examples/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. +- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. - NOTE: if it changed during the transform, you should use changed one class definition. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { From ac76e738bd9d466669721c8511f8d8bf999521fe Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 9 Dec 2020 19:54:10 +0300 Subject: [PATCH 70/73] Updated README.md and javadoc regarding comments --- .../complete/kafkatopubsub/KafkaToPubsub.java | 117 +++++++++--------- .../examples/complete/kafkatopubsub/README.md | 77 +++++++----- 2 files changed, 104 insertions(+), 90 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java index a5155428292a..5612b4a30c79 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java @@ -55,67 +55,72 @@ *
    • Kafka Topic(s) exists. *
    • The PubSub output topic exists. *
    • (Optional) An existing HashiCorp Vault secret storage + *
    • (Optional) A configured secure SSL connection for Kafka *
    * *

    Example Usage * *

    - * # Set the pipeline vars
    - * PROJECT=id-of-my-project
    - * BUCKET_NAME=my-bucket
    - *
    - * # Set containerization vars
    - * IMAGE_NAME=my-image-name
    - * TARGET_GCR_IMAGE=gcr.io/${PROJECT}/${IMAGE_NAME}
    - * BASE_CONTAINER_IMAGE=my-base-container-image
    - * TEMPLATE_PATH="gs://${BUCKET_NAME}/templates/kafka-pubsub.json"
    - *
    - * # Create bucket in the cloud storage
    - * gsutil mb gs://${BUCKET_NAME}
    - *
    - * # Go to the beam folder
    - * cd /path/to/beam
    - *
    - * FLEX TEMPLATE
    - * # Assemble uber-jar
    - * ./gradlew -p templates/kafka-to-pubsub clean shadowJar
    - *
    - * # Go to the template folder
    - * cd /path/to/beam/templates/kafka-to-pubsub
    - *
    - * # Build the flex template
    - * gcloud dataflow flex-template build ${TEMPLATE_PATH} \
    - *       --image-gcr-path "${TARGET_GCR_IMAGE}" \
    - *       --sdk-language "JAVA" \
    - *       --flex-template-base-image ${BASE_CONTAINER_IMAGE} \
    - *       --metadata-file "src/main/resources/kafka_to_pubsub_metadata.json" \
    - *       --jar "build/libs/beam-templates-kafka-to-pubsub--all.jar" \
    - *       --env FLEX_TEMPLATE_JAVA_MAIN_CLASS="org.apache.beam.templates.KafkaToPubsub"
    - *
    - * # Execute template:
    - *    API_ROOT_URL="https://dataflow.googleapis.com"
    - *    TEMPLATES_LAUNCH_API="${API_ROOT_URL}/v1b3/projects/${PROJECT}/locations/${REGION}/flexTemplates:launch"
    - *    JOB_NAME="kafka-to-pubsub-`date +%Y%m%d-%H%M%S-%N`"
    - *
    - *    time curl -X POST -H "Content-Type: application/json" \
    - *            -H "Authorization: Bearer $(gcloud auth print-access-token)" \
    - *            -d '
    - *             {
    - *                 "launch_parameter": {
    - *                     "jobName": "'$JOB_NAME'",
    - *                     "containerSpecGcsPath": "'$TEMPLATE_PATH'",
    - *                     "parameters": {
    - *                         "bootstrapServers": "broker_1:9091, broker_2:9092",
    - *                         "inputTopics": "topic1, topic2",
    - *                         "outputTopic": "projects/'$PROJECT'/topics/your-topic-name",
    - *                         "secretStoreUrl": "http(s)://host:port/path/to/credentials",
    - *                         "vaultToken": "your-token"
    - *                     }
    - *                 }
    - *             }
    - *            '
    - *            "${TEMPLATES_LAUNCH_API}"
    - * 
    + * # Gradle preparation + * + * To run this example your {@code build.gradle} file should contain the following task + * to execute the pipeline: + * {@code + * task execute (type:JavaExec) { + * main = System.getProperty("mainClass") + * classpath = sourceSets.main.runtimeClasspath + * systemProperties System.getProperties() + * args System.getProperty("exec.args", "").split() + * } + * } + * + * This task allows to run the pipeline via the following command: + * {@code + * gradle clean execute -DmainClass=org.apache.beam.examples.complete.kafkatopubsub.KafkaToPubsub \ + * -Dexec.args="--= --=" + * } + * + * # Running the pipeline + * To execute this pipeline, specify the parameters: + * + * - Kafka Bootstrap servers + * - Kafka input topics + * - Pub/Sub output topic + * - Output format + * + * in the following format: + * {@code + * --bootstrapServers=host:port \ + * --inputTopics=your-input-topic \ + * --outputTopic=projects/your-project-id/topics/your-topic-pame \ + * --outputFormat=AVRO|PUBSUB + * } + * + * Optionally, to retrieve Kafka credentials for SASL/SCRAM, + * specify a URL to the credentials in HashiCorp Vault and the vault access token: + * {@code + * --secretStoreUrl=http(s)://host:port/path/to/credentials + * --vaultToken=your-token + * } + * + * Optionally, to configure secure SSL connection between the Beam pipeline and Kafka, + * specify the parameters: + * - A path to a truststore file (it can be a local path or a GCS path, which should start with `gs://`) + * - A path to a keystore file (it can be a local path or a GCS path, which should start with `gs://`) + * - Truststore password + * - Keystore password + * - Key password + * {@code + * --truststorePath=path/to/kafka.truststore.jks + * --keystorePath=path/to/kafka.keystore.jks + * --truststorePassword=your-truststore-password + * --keystorePassword=your-keystore-password + * --keyPassword=your-key-password + * } + * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: + * {@code + * --runner=YOUR_SELECTED_RUNNER + * } * *

    Example Avro usage * diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md index 283ca9dd281f..48525ed00645 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md @@ -49,42 +49,38 @@ In a simple scenario, the example will create an Apache Beam pipeline that will ## Getting Started -This section describes what is needed to get the exaple up and running. -- Assembling the Uber-JAR +This section describes what is needed to get the example up and running. +- Gradle preparation - Local execution -- Google Dataflow Template - - Set up the environment - - Creating the Dataflow Flex Template - - Create a Dataflow job to ingest data using the template -- Avro format transferring. +- Running as a Dataflow Template +- Supported Output Formats + - PubSubMessage + - AVRO - E2E tests (TBD) -## Assembling the Uber-JAR +## Gradle preparation -To run this example the Java project should be built into -an Uber JAR file. - -Navigate to the Beam folder: +To run this example your `build.gradle` file should contain the following task +to execute the pipeline: ``` -cd /path/to/beam +task execute (type:JavaExec) { + main = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} ``` -In order to create Uber JAR with Gradle, [Shadow plugin](https://github.com/johnrengelman/shadow) -is used. It creates the `shadowJar` task that builds the Uber JAR: +This task allows to run the pipeline via the following command: +```bash +gradle clean execute -DmainClass=org.apache.beam.examples.complete.kafkatopubsub.KafkaToPubsub \ + -Dexec.args="--= --=" ``` -./gradlew -p examples/java clean shadowJar -``` - -ℹ️ An **Uber JAR** - also known as **fat JAR** - is a single JAR file that contains -both target package *and* all its dependencies. - -The result of the `shadowJar` task execution is a `.jar` file that is generated -under the `build/libs/` folder in examples/java directory. -## Local execution -To execute this pipeline locally, specify the parameters: +## Running the pipeline +To execute this pipeline, specify the parameters: - Kafka Bootstrap servers - Kafka input topics - Pub/Sub output topic @@ -121,24 +117,37 @@ By default this will run the pipeline locally with the DirectRunner. To change t ```bash --runner=YOUR_SELECTED_RUNNER ``` -See examples/java/README.md for steps and examples to configure different runners. +See the [documentation](http://beam.apache.org/get-started/quickstart/) and the [Examples +README](../../../../../../../../../README.md) for more +information about how to run this example. + +## Running as a Dataflow Template + +This example also exists as Google Dataflow Template, which you can build and run using Google Cloud Platform. +See its [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/kafka-to-pubsub/README.md) for more information. + +## Supported Output Formats + +This pipeline can output data in a format of PubSubMessage or AVRO. -## Google Dataflow Execution +### PubSubMessage -This example also exists as Google Dataflow Template, see its [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/kafka-to-pubsub/README.md) for more information. +This example supports PubSubMessage format for output out-of-the-box. +No additional changes are required. -## AVRO format transferring. +### AVRO -This example contains an example demonstrating AVRO format support: +This example contains an example demonstrating AVRO format support, +the following steps should be done to provide it: - Define custom Class to deserialize AVRO from Kafka [provided in example] - Create custom data serialization in Apache Beam - Serialize data to AVRO in Pub/Sub [provided in example]. To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [AvroDataClass](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClass.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](avro/AvroDataClass.java). Just define necessary fields. +- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. ```java return KafkaIO.read() ... @@ -147,7 +156,7 @@ return KafkaIO.read() ... ``` - [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. -- Modify the write step in the [KafkaToPubsub class](src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. +- Modify the write step in the [KafkaToPubsub class](KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. - NOTE: if it changed during the transform, you should use changed one class definition. ```java if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { From b0c44289db70c7a4dc2b6f0f6275c62bc2214c86 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 9 Dec 2020 20:16:07 +0300 Subject: [PATCH 71/73] README.md stylefix --- .../examples/complete/kafkatopubsub/README.md | 94 ++++++++++++------- 1 file changed, 61 insertions(+), 33 deletions(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md index 48525ed00645..13f2d3a5f593 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/README.md @@ -19,24 +19,32 @@ # Apache Beam pipeline example to ingest data from Apache Kafka to Google Cloud Pub/Sub -This directory contains an [Apache Beam](https://beam.apache.org/) pipeline example that creates a pipeline -to read data from a single or multiple topics from +This directory contains an [Apache Beam](https://beam.apache.org/) pipeline example that creates a pipeline to read data +from a single or multiple topics from [Apache Kafka](https://kafka.apache.org/) and write data into a single topic in [Google Cloud Pub/Sub](https://cloud.google.com/pubsub). Supported data formats: + - Serializable plaintext formats, such as JSON - [PubSubMessage](https://cloud.google.com/pubsub/docs/reference/rest/v1/PubsubMessage). Supported input source configurations: + - Single or multiple Apache Kafka bootstrap servers - Apache Kafka SASL/SCRAM authentication over plaintext or SSL connection - Secrets vault service [HashiCorp Vault](https://www.vaultproject.io/). Supported destination configuration: + - Single Google Cloud Pub/Sub topic. -In a simple scenario, the example will create an Apache Beam pipeline that will read messages from a source Kafka server with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The example supports using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL authenticaton over SSL the example will need an SSL certificate location and access to a secrets vault service with Kafka username and password, currently supporting HashiCorp Vault. +In a simple scenario, the example will create an Apache Beam pipeline that will read messages from a source Kafka server +with a source topic, and stream the text messages into specified Pub/Sub destination topic. Other scenarios may need +Kafka SASL/SCRAM authentication, that can be performed over plain text or SSL encrypted connection. The example supports +using a single Kafka user account to authenticate in the provided source Kafka servers and topics. To support SASL +authenticaton over SSL the example will need an SSL certificate location and access to a secrets vault service with +Kafka username and password, currently supporting HashiCorp Vault. ## Requirements @@ -50,18 +58,18 @@ In a simple scenario, the example will create an Apache Beam pipeline that will ## Getting Started This section describes what is needed to get the example up and running. + - Gradle preparation - Local execution - Running as a Dataflow Template -- Supported Output Formats - - PubSubMessage - - AVRO +- Supported Output Formats + - PubSubMessage + - AVRO - E2E tests (TBD) ## Gradle preparation -To run this example your `build.gradle` file should contain the following task -to execute the pipeline: +To run this example your `build.gradle` file should contain the following task to execute the pipeline: ``` task execute (type:JavaExec) { @@ -80,32 +88,39 @@ gradle clean execute -DmainClass=org.apache.beam.examples.complete.kafkatopubsub ``` ## Running the pipeline + To execute this pipeline, specify the parameters: + - Kafka Bootstrap servers - Kafka input topics - Pub/Sub output topic - Output format in the following format: + ```bash --bootstrapServers=host:port \ --inputTopics=your-input-topic \ --outputTopic=projects/your-project-id/topics/your-topic-pame \ --outputFormat=AVRO|PUBSUB ``` -Optionally, to retrieve Kafka credentials for SASL/SCRAM, -specify a URL to the credentials in HashiCorp Vault and the vault access token: + +Optionally, to retrieve Kafka credentials for SASL/SCRAM, specify a URL to the credentials in HashiCorp Vault and the +vault access token: + ```bash --secretStoreUrl=http(s)://host:port/path/to/credentials --vaultToken=your-token ``` -Optionally, to configure secure SSL connection between the Beam pipeline and Kafka, -specify the parameters: + +Optionally, to configure secure SSL connection between the Beam pipeline and Kafka, specify the parameters: + - A path to a truststore file (it can be a local path or a GCS path, which should start with `gs://`) - A path to a keystore file (it can be a local path or a GCS path, which should start with `gs://`) - Truststore password - Keystore password - Key password + ```bash --truststorePath=path/to/kafka.truststore.jks --keystorePath=path/to/kafka.keystore.jks @@ -113,60 +128,73 @@ specify the parameters: --keystorePassword=your-keystore-password --keyPassword=your-key-password ``` + By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: + ```bash --runner=YOUR_SELECTED_RUNNER ``` -See the [documentation](http://beam.apache.org/get-started/quickstart/) and the [Examples -README](../../../../../../../../../README.md) for more -information about how to run this example. + +See the [documentation](http://beam.apache.org/get-started/quickstart/) and +the [Examples README](../../../../../../../../../README.md) for more information about how to run this example. ## Running as a Dataflow Template -This example also exists as Google Dataflow Template, which you can build and run using Google Cloud Platform. -See its [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/kafka-to-pubsub/README.md) for more information. +This example also exists as Google Dataflow Template, which you can build and run using Google Cloud Platform. See +its [README.md](https://github.com/GoogleCloudPlatform/DataflowTemplates/blob/master/v2/kafka-to-pubsub/README.md) for +more information. ## Supported Output Formats -This pipeline can output data in a format of PubSubMessage or AVRO. +This pipeline can output data in a format of PubSubMessage or AVRO. ### PubSubMessage -This example supports PubSubMessage format for output out-of-the-box. -No additional changes are required. +This example supports PubSubMessage format for output out-of-the-box. No additional changes are required. ### AVRO -This example contains an example demonstrating AVRO format support, -the following steps should be done to provide it: +This example contains an example demonstrating AVRO format support, the following steps should be done to provide it: + - Define custom Class to deserialize AVRO from Kafka [provided in example] - Create custom data serialization in Apache Beam - Serialize data to AVRO in Pub/Sub [provided in example]. To use this example in the specific case, follow these steps: -- Create your own class to describe AVRO schema. As an example use [AvroDataClass](avro/AvroDataClass.java). Just define necessary fields. -- Create your own Avro Deserializer class. As an example use [AvroDataClassKafkaAvroDeserializer class](avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put your own Schema class as the necessary types. -- Modify the [FormatTransform.readAvrosFromKafka method](transforms/FormatTransform.java). Put your Schema class and Deserializer to the related parameter. +- Create your own class to describe AVRO schema. As an example use [AvroDataClass](avro/AvroDataClass.java). Just define + necessary fields. +- Create your own Avro Deserializer class. As an example + use [AvroDataClassKafkaAvroDeserializer class](avro/AvroDataClassKafkaAvroDeserializer.java). Just rename it, and put + your own Schema class as the necessary types. +- Modify the [FormatTransform.readAvrosFromKafka method](transforms/FormatTransform.java). Put your Schema class and + Deserializer to the related parameter. + ```java return KafkaIO.read() ... .withValueDeserializerAndCoder( - AvroDataClassKafkaAvroDeserializer.class, AvroCoder.of(AvroDataClass.class)) // put your classes here + AvroDataClassKafkaAvroDeserializer.class,AvroCoder.of(AvroDataClass.class)) // put your classes here ... ``` -- [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if it necessary in your case. -- Modify the write step in the [KafkaToPubsub class](KafkaToPubsub.java) by putting your Schema class to "writeAvrosToPubSub" step. + +- [OPTIONAL TO IMPLEMENT] Add [Beam Transform](https://beam.apache.org/documentation/programming-guide/#transforms) if + it necessary in your case. +- Modify the write step in the [KafkaToPubsub class](KafkaToPubsub.java) by putting your Schema class to " + writeAvrosToPubSub" step. - NOTE: if it changed during the transform, you should use changed one class definition. + ```java -if (options.getOutputFormat() == FormatTransform.FORMAT.AVRO) { - ... - .apply("writeAvrosToPubSub", PubsubIO.writeAvros(AvroDataClass.class)); // put your SCHEMA class here +if(options.getOutputFormat()==FormatTransform.FORMAT.AVRO){ + ... + .apply("writeAvrosToPubSub",PubsubIO.writeAvros(AvroDataClass.class)); // put your SCHEMA class here - } + } ``` ## End to end tests + TBD -_Note: The Kafka to Pub/Sub job executed with a distributed runner supports SSL configuration with the certificate located only in GCS._ +_Note: The Kafka to Pub/Sub job executed with a distributed runner supports SSL configuration with the certificate +located only in GCS._ From 3931fba06323d0e238355e8c5538c61e6a26310a Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 9 Dec 2020 20:33:04 +0300 Subject: [PATCH 72/73] Added link to KafkaToPubsub example into complete/README.md --- .../main/java/org/apache/beam/examples/complete/README.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md index 3f4842a5c7a7..4840dfe4437c 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md @@ -32,6 +32,11 @@ This directory contains end-to-end example pipelines that perform complex data p Pub/Sub topic, splits each line into individual words, capitalizes those words, and writes the output to a BigQuery table. +

  • KafkaToPubsub + — A streaming pipeline example that creates a pipeline to read data + from a single or multiple topics from Apache Kafka and write data into a single topic + in Google Cloud Pub/Sub. +
  • TfIdf — An example that computes a basic TF-IDF search table for a directory or Cloud Storage prefix. Demonstrates joining data, side inputs, and logging. From 21d8b36652b9be13ba17be1c2ae8ef6e3c351472 Mon Sep 17 00:00:00 2001 From: Artur Khanin Date: Wed, 9 Dec 2020 20:47:58 +0300 Subject: [PATCH 73/73] Stylefix --- .../src/main/java/org/apache/beam/examples/complete/README.md | 2 +- .../beam/examples/complete/kafkatopubsub/KafkaToPubsub.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md index 4840dfe4437c..b74ea44554d5 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/README.md +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/README.md @@ -35,7 +35,7 @@ This directory contains end-to-end example pipelines that perform complex data p
  • KafkaToPubsub — A streaming pipeline example that creates a pipeline to read data from a single or multiple topics from Apache Kafka and write data into a single topic - in Google Cloud Pub/Sub. + in Google Cloud Pub/Sub.
  • TfIdf — An example that computes a basic TF-IDF search table for a directory or diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java index 5612b4a30c79..8b4e2b305927 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/kafkatopubsub/KafkaToPubsub.java @@ -121,6 +121,7 @@ * {@code * --runner=YOUR_SELECTED_RUNNER * } + *
  • * *

    Example Avro usage *