From cf70453bf6c6818955e1edd433cf11cfa8131fba Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 5 Oct 2022 18:02:18 +0400 Subject: [PATCH 01/21] Add complete examples for CdapIO --- examples/java/build.gradle | 20 +- .../cdap/CdapHubspotStreamingToTxt.java | 145 +++++++++++++ .../complete/cdap/CdapHubspotToTxt.java | 142 +++++++++++++ .../complete/cdap/CdapSalesforceToTxt.java | 137 ++++++++++++ .../complete/cdap/CdapServiceNowToTxt.java | 143 +++++++++++++ .../complete/cdap/CdapZendeskToTxt.java | 140 +++++++++++++ .../beam/examples/complete/cdap/README.md | 197 ++++++++++++++++++ .../complete/cdap/TxtToCdapHubspot.java | 129 ++++++++++++ .../complete/cdap/TxtToCdapSalesforce.java | 153 ++++++++++++++ .../cdap/options/CdapHubspotOptions.java | 64 ++++++ .../cdap/options/CdapSalesforceOptions.java | 75 +++++++ .../options/CdapSalesforceSinkOptions.java | 71 +++++++ .../options/CdapSalesforceSourceOptions.java | 31 +++ .../cdap/options/CdapServiceNowOptions.java | 100 +++++++++ .../cdap/options/CdapZendeskOptions.java | 103 +++++++++ .../complete/cdap/options/package-info.java | 20 ++ .../examples/complete/cdap/package-info.java | 20 ++ .../cdap/transforms/FormatInputTransform.java | 151 ++++++++++++++ .../transforms/FormatOutputTransform.java | 80 +++++++ .../cdap/transforms/package-info.java | 20 ++ .../complete/cdap/utils/CsvRecordCoder.java | 48 +++++ .../complete/cdap/utils/JsonElementCoder.java | 49 +++++ .../utils/PluginConfigOptionsConverter.java | 127 +++++++++++ .../cdap/utils/StructuredRecordUtils.java | 60 ++++++ .../complete/cdap/utils/package-info.java | 20 ++ .../example-txt-to-cdap-hubspot-companies.txt | 20 ++ .../example-txt-to-cdap-salesforce-custom.txt | 20 ++ sdks/java/io/cdap/build.gradle | 4 +- .../apache/beam/sdk/io/cdap/MappingUtils.java | 5 + .../org/apache/beam/sdk/io/cdap/Plugin.java | 2 +- .../sdk/io/cdap/context/BatchContextImpl.java | 51 ++++- .../cdap/context/FailureCollectorWrapper.java | 18 ++ .../context/FailureCollectorWrapperTest.java | 24 +++ sdks/java/io/hadoop-format/build.gradle | 2 - .../HadoopFormatIOSequenceFileTest.java | 61 +++++- 35 files changed, 2438 insertions(+), 14 deletions(-) create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java create mode 100644 examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java create mode 100644 examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt create mode 100644 examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt diff --git a/examples/java/build.gradle b/examples/java/build.gradle index aa51dcfeae85..82c3792f7c40 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -60,9 +60,19 @@ dependencies { implementation project(":sdks:java:extensions:python") implementation project(":sdks:java:io:google-cloud-platform") implementation project(":sdks:java:io:kafka") + implementation project(":sdks:java:io:cdap") + implementation project(":sdks:java:io:hadoop-common") implementation project(":sdks:java:extensions:ml") implementation library.java.avro implementation library.java.bigdataoss_util + implementation library.java.cdap_api + implementation library.java.cdap_api_commons + implementation library.java.cdap_etl_api + implementation library.java.cdap_hydrator_common + implementation library.java.cdap_plugin_hubspot + implementation library.java.cdap_plugin_service_now + implementation library.java.cdap_plugin_zendesk + implementation library.java.cdap_plugin_salesforce implementation library.java.google_api_client implementation library.java.google_api_services_bigquery implementation library.java.google_api_services_pubsub @@ -72,6 +82,7 @@ dependencies { implementation library.java.google_code_gson implementation library.java.google_http_client implementation library.java.google_oauth_client + implementation library.java.hadoop_common implementation library.java.jackson_databind implementation library.java.joda_time implementation library.java.protobuf_java @@ -164,4 +175,11 @@ task wordCount(type:JavaExec) { classpath = sourceSets.main.runtimeClasspath systemProperties = System.getProperties() args = ["--output=/tmp/ouput.txt"] -} \ No newline at end of file +} + +task executeCdap (type:JavaExec) { + mainClass = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java new file mode 100644 index 000000000000..682ac8bc2888 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -0,0 +1,145 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import com.google.gson.JsonElement; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; +import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.hadoop.io.NullWritable; +import org.joda.time.Duration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapHubspotStreamingToTxt} pipeline is a streaming pipeline which ingests data in JSON + * format from CDAP Hubspot, and outputs the resulting records to .txt file. Hubspot parameters and + * output .txt file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapHubspotStreamingToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --apikey=your-api-key \
+ * --referenceName=your-reference-name \
+ * --objectType=Contacts \
+ * --txtFilePath=your-path-to-output-file
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapHubspotStreamingToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapHubspotStreamingToTxt.class); + public static final int SECONDS_TO_READ = 30; + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapHubspotOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from CDAP Hubspot and writes it to .txt file. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, true); + LOG.info("Starting Cdap-Hubspot-streaming-to-txt pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages in from Cdap Hubspot + * 2) Extract values only + * 3) Write successful records to .txt file + */ + pipeline.getCoderRegistry().registerCoderForClass(JsonElement.class, JsonElementCoder.of()); + + pipeline + .apply( + "readFromCdapHubspotStreaming", + FormatInputTransform.readFromCdapHubspotStreaming(paramsMap)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) + .apply( + "globalwindow", + Window.>into(new GlobalWindows()) + .triggering( + Repeatedly.forever( + AfterProcessingTime.pastFirstElementInPane() + .plusDelayOf(Duration.standardSeconds(SECONDS_TO_READ)))) + .discardingFiredPanes()) + .apply(Values.create()) + .apply("writeToTxt", TextIO.write().withWindowedWrites().to(options.getTxtFilePath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java new file mode 100644 index 000000000000..dc7df5f3c87b --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java @@ -0,0 +1,142 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import com.google.gson.JsonElement; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; +import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapValues; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapHubspotToTxt} pipeline is a batch pipeline which ingests data in JSON format from + * CDAP Hubspot, and outputs the resulting records to .txt file. Hubspot parameters and output .txt + * file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapHubspotToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --apikey=your-api-key \
+ * --referenceName=your-reference-name \
+ * --objectType=Contacts \
+ * --txtFilePath=your-path-to-output-file
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapHubspotToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapHubspotToTxt.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapHubspotOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from CDAP Hubspot and writes it to .txt file. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); + LOG.info("Starting Cdap-Hubspot-to-txt pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages in from Cdap Hubspot + * 2) Extract values only + * 3) Write successful records to .txt file + */ + pipeline.getCoderRegistry().registerCoderForClass(JsonElement.class, JsonElementCoder.of()); + + pipeline + .apply("readFromCdapHubspot", FormatInputTransform.readFromCdapHubspot(paramsMap)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), JsonElementCoder.of())) + .apply( + MapValues.into(TypeDescriptors.strings()) + .via( + jsonElement -> { + if (jsonElement == null) { + return "{}"; + } + return jsonElement.toString(); + })) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) + .apply(Values.create()) + .apply("writeToTxt", TextIO.write().to(options.getTxtFilePath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java new file mode 100644 index 000000000000..58a5b34a0438 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java @@ -0,0 +1,137 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import io.cdap.cdap.api.data.schema.Schema; +import java.util.LinkedHashMap; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSourceOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapValues; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapSalesforceToTxt} pipeline is a batch pipeline which ingests data in JSON format + * from CDAP Salesforce, and outputs the resulting records to .txt file. Salesforce parameters and + * output txt file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapSalesforceToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --username=your-user-name\
+ * --password=your-password \
+ * --securityToken=your-token \
+ * --consumerKey=your-key \
+ * --consumerSecret=your-secret \
+ * --loginUrl=your-login-url \
+ * --sObjectName=object-name \
+ * --referenceName=your-reference-name \
+ * --txtFilePath=your-path-to-file
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapSalesforceToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapSalesforceToTxt.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapSalesforceSourceOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(CdapSalesforceSourceOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from CDAP Salesforce plugin. + * + * @param options arguments to the pipeline + */ + @SuppressWarnings("rawtypes") + public static PipelineResult run(Pipeline pipeline, CdapSalesforceSourceOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.salesforceSourceOptionsToParamsMap(options); + LOG.info("Starting Cdap-Salesforce pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages from Cdap Salesforce + * 2) Extract values only + * 3) Write successful records to .txt file + */ + + pipeline + .apply("readFromCdapSalesforce", FormatInputTransform.readFromCdapSalesforce(paramsMap)) + .setCoder( + KvCoder.of( + SerializableCoder.of(Schema.class), SerializableCoder.of(LinkedHashMap.class))) + .apply(MapValues.into(TypeDescriptors.strings()).via(LinkedHashMap::toString)) + .setCoder(KvCoder.of(SerializableCoder.of(Schema.class), StringUtf8Coder.of())) + .apply(Values.create()) + .apply("writeToTxt", TextIO.write().to(options.getTxtFilePath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java new file mode 100644 index 000000000000..88a7076659a4 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import io.cdap.cdap.api.data.format.StructuredRecord; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapServiceNowOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.examples.complete.cdap.utils.StructuredRecordUtils; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapValues; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapServiceNowToTxt} pipeline is a batch pipeline which ingests data in JSON format + * from CDAP ServiceNow, and outputs the resulting records to .txt file. ServiceNow parameters and + * output txt file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapServiceNowToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --clientId=your-client-id \
+ * --clientSecret=your-client-secret \
+ * --user=your-user \
+ * --password=your-password \
+ * --restApiEndpoint=your-endpoint \
+ * --queryMode=Table \
+ * --tableName=your-table \
+ * --valueType=Actual \
+ * --referenceName=your-reference-name \
+ * --outputTxtFilePath=your-path-to-file
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapServiceNowToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapServiceNowToTxt.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapServiceNowOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapServiceNowOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from CDAP ServiceNow and writes it to .txt file. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapServiceNowOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.serviceNowOptionsToParamsMap(options); + LOG.info("Starting Cdap-ServiceNow pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages in from Cdap ServiceNow + * 2) Extract values only + * 3) Write successful records to .txt file + */ + + pipeline + .apply("readFromCdapServiceNow", FormatInputTransform.readFromCdapServiceNow(paramsMap)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), + SerializableCoder.of(StructuredRecord.class))) + .apply( + MapValues.into(TypeDescriptors.strings()) + .via(StructuredRecordUtils::structuredRecordToString)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) + .apply(Values.create()) + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java new file mode 100644 index 000000000000..d01b19534d99 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java @@ -0,0 +1,140 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import static org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform.readFromCdapZendesk; + +import io.cdap.cdap.api.data.format.StructuredRecord; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapZendeskOptions; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.examples.complete.cdap.utils.StructuredRecordUtils; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.SerializableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapValues; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.values.TypeDescriptors; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapZendeskToTxt} pipeline is a batch pipeline which ingests data in JSON format from + * CDAP Zendesk, and outputs the resulting records to .txt file. Zendesk parameters and output txt + * file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.zendesk.CdapZendeskToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --zendeskBaseUrl=your-url \
+ * --adminEmail=your-email \
+ * --apiToken=your-token \
+ * --objectsToPull=Groups \
+ * --referenceName=your-reference-name \
+ * --outputTxtFilePath=your-path-to-file
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapZendeskToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapZendeskToTxt.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapZendeskOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapZendeskOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads message from CDAP and writes it to .txt file. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapZendeskOptions options) { + Map pluginConfigParams = + PluginConfigOptionsConverter.zendeskOptionsToParamsMap(options); + LOG.info("Starting Cdap-Zendesk-To-Txt pipeline with parameters: {}", pluginConfigParams); + + /* + * Steps: + * 1) Read messages in from Cdap Zendesk + * 2) Extract values only + * 3) Write successful records to .txt file + */ + + pipeline + .apply("readFromCdapZendesk", readFromCdapZendesk(pluginConfigParams)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), + SerializableCoder.of(StructuredRecord.class))) + .apply( + MapValues.into(TypeDescriptors.strings()) + .via(StructuredRecordUtils::structuredRecordToString)) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) + .apply(Values.create()) + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md new file mode 100644 index 000000000000..09ba15207dd3 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -0,0 +1,197 @@ + + +# Apache Beam pipeline examples to ingest data from CDAP plugin to TXT file + +This directory contains set of [Apache Beam](https://beam.apache.org/) pipeline examples that create a pipeline to read data +from a [CDAP](https://cdap.atlassian.net/wiki/spaces/DOCS/overview?homepageId=379748484) plugin +and write data into .txt file (and vice versa). + +Supported CDAP plugins: + +- [Hubspot](https://github.com/data-integrations/hubspot) +- [Salesforce](https://github.com/data-integrations/salesforce) +- [ServiceNow](https://github.com/data-integrations/servicenow-plugins) +- [Zendesk](https://github.com/data-integrations/zendesk) + +## Gradle preparation + +To run this example your `build.gradle` file should contain the following task to execute the pipeline: + +``` +task executeCdap (type:JavaExec) { + mainClass = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} +``` + +## Running the CdapHubspotToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapHubspotToTxt \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --apikey=your-api-key \ + --referenceName=your-reference-name \ + --objectType=Contacts \ + --txtFilePath=your-path-to-output-file +``` + +## Running the CdapHubspotStreamingToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapHubspotStreamingToTxt \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --apikey=your-api-key \ + --referenceName=your-reference-name \ + --objectType=Contacts \ + --txtFilePath=your-path-to-output-file +``` + +## Running the TxtToCdapHubspot pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtToCdapHubspot \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --apikey=your-api-key \ + --referenceName=your-reference-name \ + --objectType=your-object-type \ + --txtFilePath=your-path-to-input-file \ + --locksDirPath=your-path-to-locks-dir +``` + +## Running the CdapServiceNowToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapServiceNowToTxt \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --clientId=your-client-id \ + --clientSecret=your-client-secret \ + --user=your-user \ + --password=your-password \ + --restApiEndpoint=your-endpoint \ + --queryMode=Table \ + --tableName=your-table \ + --valueType=Actual \ + --referenceName=your-reference-name \ + --outputTxtFilePath=your-path-to-file +``` + +## Running the CdapSalesforceToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapSalesforceToTxt \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --username=your-user-name\ + --password=your-password \ + --securityToken=your-token \ + --consumerKey=your-key \ + --consumerSecret=your-secret \ + --loginUrl=your-login-url \ + --sObjectName=object-name \ + --referenceName=your-reference-name \ + --txtFilePath=your-path-to-file +``` + +## Running the TxtToCdapSalesforce pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtToCdapSalesforce \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --username=your-user-name\ + --password=your-password \ + --securityToken=your-token \ + --consumerKey=your-key \ + --consumerSecret=your-secret \ + --loginUrl=your-login-url \ + --sObject=CustomObject__c \ + --referenceName=your-reference-name \ + --txtFilePath=your-path-to-file \ + --operation=Insert \ + --errorHandling=Stop on error \ + --maxRecordsPerBatch=10 \ + --maxBytesPerBatch=9999999 \ + --locksDirPath=your-path +``` + +## Running the CdapZendeskToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapZendeskToTxt \ + -Dexec.args="--= --=" +``` + +To execute this pipeline, specify the parameters in the following format: + +```bash + --zendeskBaseUrl=zendesk-url-key-followed-by-/%s/%s (example: https://support.zendesk.com/%s/%s) \ + --adminEmail=your-admin-admin-email \ + --apiToken=your-api-token \ + --subdomains=your-subdomains (example: api/v2) \ + --maxRetryCount=your-max-retry-count \ + --maxRetryWait=your-max-retry-wait \ + --maxRetryJitterWait=your-max-retry-jitter-wait \ + --connectTimeout=your-connection-timeout \ + --readTimeout=your-read-timeout \ + --objectsToPull=your-objects-to-pull (example: Groups) \ + --outputTxtFilePath=your-path-to-file +``` diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java new file mode 100644 index 000000000000..7c26b504d936 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java @@ -0,0 +1,129 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatOutputTransform; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link TxtToCdapHubspot} pipeline is a batch pipeline which ingests data in JSON format from + * .txt file, and outputs the resulting records to Hubspot. Hubspot parameters and input .txt file + * path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtToCdapHubspot \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --apikey=your-api-key \
+ * --referenceName=your-reference-name \
+ * --objectType=your-object-type \
+ * --txtFilePath=your-path-to-input-file \
+ * --locksDirPath=your-path-to-locks-dir
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class TxtToCdapHubspot { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(TxtToCdapHubspot.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapHubspotOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + + checkStateNotNull(options.getLocksDirPath(), "locksDirPath can not be null!"); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from .txt file and writes it to CDAP Hubspot. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); + LOG.info("Starting Txt-to-Cdap-Hubspot pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages in from .txt file + * 2) Map to KV + * 3) Write successful records to Cdap Hubspot + */ + + pipeline + .apply("readFromTxt", TextIO.read().from(options.getTxtFilePath())) + .apply( + MapElements.into(new TypeDescriptor>() {}) + .via(json -> KV.of(NullWritable.get(), json))) + .apply( + "writeToCdapHubspot", + FormatOutputTransform.writeToCdapHubspot(paramsMap, options.getLocksDirPath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java new file mode 100644 index 000000000000..1386eb0bf092 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import com.google.gson.Gson; +import io.cdap.plugin.salesforce.plugin.sink.batch.CSVRecord; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSinkOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatOutputTransform; +import org.apache.beam.examples.complete.cdap.utils.CsvRecordCoder; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.MapElements; +import org.apache.beam.sdk.values.KV; +import org.apache.beam.sdk.values.TypeDescriptor; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link TxtToCdapSalesforce} pipeline is a batch pipeline which ingests data from .txt file, + * and outputs the resulting records to Salesforce. Salesforce parameters and input .txt file path + * are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtToCdapSalesforce \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --username=your-user-name\
+ * --password=your-password \
+ * --securityToken=your-token \
+ * --consumerKey=your-key \
+ * --consumerSecret=your-secret \
+ * --loginUrl=your-login-url \
+ * --sObject=CustomObject__c \
+ * --referenceName=your-reference-name \
+ * --txtFilePath=your-path-to-file \
+ * --maxRecordsPerBatch=10 \
+ * --maxBytesPerBatch=9999999 \
+ * --operation=Insert \
+ * --errorHandling=Stop on error
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class TxtToCdapSalesforce { + + private static final Gson GSON = new Gson(); + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(TxtToCdapSalesforce.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapSalesforceSinkOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapSalesforceSinkOptions.class); + + checkStateNotNull(options.getLocksDirPath(), "locksDirPath can not be null!"); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from .txt file and writes it to CDAP Salesforce. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run(Pipeline pipeline, CdapSalesforceSinkOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.salesforceSinkOptionsToParamsMap(options); + LOG.info("Starting Txt-to-Cdap-Salesforce pipeline with parameters: {}", paramsMap); + + /* + * Steps: + * 1) Read messages in from .txt file + * 2) Map to KV + * 3) Write successful records to Cdap Salesforce + */ + + pipeline + .apply("readFromTxt", TextIO.read().from(options.getTxtFilePath())) + .apply( + MapElements.into(new TypeDescriptor>() {}) + .via( + json -> { + CSVRecord csvRecord = GSON.fromJson(json, CSVRecord.class); + return KV.of(NullWritable.get(), csvRecord); + })) + .setCoder( + KvCoder.of(NullableCoder.of(WritableCoder.of(NullWritable.class)), CsvRecordCoder.of())) + .apply( + "writeToCdapSalesforce", + FormatOutputTransform.writeToCdapSalesforce( + PluginConfigOptionsConverter.salesforceSinkOptionsToParamsMap(options), + options.getLocksDirPath())); + + return pipeline.run(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java new file mode 100644 index 000000000000..b8628dd0ad01 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.common.Constants; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link CdapHubspotOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ +public interface CdapHubspotOptions extends PipelineOptions { + + @Description("Hubspot api server url. If not specified then the default url will be used.") + String getApiServerUrl(); + + void setApiServerUrl(String apiServerUrl); + + @Validation.Required + @Description("Hubspot OAuth2 API Key.") + String getApiKey(); + + void setApiKey(String apiKey); + + @Validation.Required + @Description("Name of object to pull from Hubspot (e.g. Contacts).") + String getObjectType(); + + void setObjectType(String objectType); + + @Validation.Required + @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) + String getReferenceName(); + + void setReferenceName(String referenceName); + + @Validation.Required + @Description("Path to input/output .txt file.") + String getTxtFilePath(); + + void setTxtFilePath(String txtFilePath); + + @Description("Locks directory path where locks will be stored.") + String getLocksDirPath(); + + void setLocksDirPath(String locksDirPath); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java new file mode 100644 index 000000000000..da83d92629e7 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java @@ -0,0 +1,75 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.common.Constants; +import io.cdap.plugin.salesforce.SalesforceConstants; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +public interface CdapSalesforceOptions extends PipelineOptions { + + @Validation.Required + @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) + String getReferenceName(); + + void setReferenceName(String referenceName); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_USERNAME) + String getUsername(); + + void setUsername(String username); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_PASSWORD) + String getPassword(); + + void setPassword(String password); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_SECURITY_TOKEN) + String getSecurityToken(); + + void setSecurityToken(String securityToken); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_CONSUMER_KEY) + String getConsumerKey(); + + void setConsumerKey(String consumerKey); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_CONSUMER_SECRET) + String getConsumerSecret(); + + void setConsumerSecret(String consumerSecret); + + @Validation.Required + @Description(SalesforceConstants.PROPERTY_LOGIN_URL) + String getLoginUrl(); + + void setLoginUrl(String loginUrl); + + @Validation.Required + @Description("Path to input/output .txt file.") + String getTxtFilePath(); + + void setTxtFilePath(String txtFilePath); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java new file mode 100644 index 000000000000..16af4621898d --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +public interface CdapSalesforceSinkOptions extends CdapSalesforceOptions { + + @Validation.Required + @Description( + "Strategy used to handle erroneous records.\n" + + "SKIP - Ignores erroneous records.\n" + + "STOP - Fails pipeline due to erroneous record.") + String getErrorHandling(); + + void setErrorHandling(String errorHandling); + + @Validation.Required + @Description( + "Maximum size in bytes of a batch of records when writing to Salesforce. " + + "This value cannot be greater than 10,000,000.") + String getMaxBytesPerBatch(); + + void setMaxBytesPerBatch(String maxBytesPerBatch); + + @Validation.Required + @Description( + "Maximum number of records to include in a batch when writing to Salesforce." + + "This value cannot be greater than 10,000.") + String getMaxRecordsPerBatch(); + + void setMaxRecordsPerBatch(String maxRecordsPerBatch); + + @Validation.Required + @Description( + "Operation used for sinking data into Salesforce.\n" + + "Insert - adds records.\n" + + "Upsert - upserts the records. Salesforce will decide if sObjects " + + "are the same using external id field.\n" + + "Update - updates existing records based on Id field.") + String getOperation(); + + void setOperation(String operation); + + @Validation.Required + @Description("Salesforce object name to insert records into.") + String getsObject(); + + void setsObject(String sObject); + + @Description("Locks directory path where locks will be stored.") + String getLocksDirPath(); + + void setLocksDirPath(String locksDirPath); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java new file mode 100644 index 000000000000..0dc7fe152fea --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +public interface CdapSalesforceSourceOptions extends CdapSalesforceOptions { + + @Validation.Required + @Description(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME) + String getSObjectName(); + + void setSObjectName(String sObjectName); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java new file mode 100644 index 000000000000..f197971a502a --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.common.Constants; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link CdapServiceNowOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ +public interface CdapServiceNowOptions extends PipelineOptions { + + @Validation.Required + @Description("The Client ID for ServiceNow Instance.") + String getClientId(); + + void setClientId(String clientId); + + @Validation.Required + @Description("The Client Secret for ServiceNow Instance.") + String getClientSecret(); + + void setClientSecret(String clientSecret); + + @Validation.Required + @Description("The user name for ServiceNow Instance.") + String getUser(); + + void setUser(String user); + + @Validation.Required + @Description("The password for ServiceNow Instance.") + String getPassword(); + + void setPassword(String password); + + @Validation.Required + @Description( + "The REST API Endpoint for ServiceNow Instance. For example, https://instance.service-now.com") + String getRestApiEndpoint(); + + void setRestApiEndpoint(String restApiEndpoint); + + @Validation.Required + @Description( + "Mode of query. The mode can be one of two values: " + + "`Reporting` - will allow user to choose application for which data will be fetched for all tables, " + + "`Table` - will allow user to enter table name for which data will be fetched.") + String getQueryMode(); + + void setQueryMode(String queryMode); + + @Validation.Required + @Description( + "The name of the ServiceNow table from which data to be fetched. Note, the Table name value " + + "will be ignored if the Mode is set to `Reporting`.") + String getTableName(); + + void setTableName(String tableName); + + @Validation.Required + @Description( + "The type of values to be returned." + + "`Actual` - will fetch the actual values from the ServiceNow tables" + + "`Display` - will fetch the display values from the ServiceNow tables." + + "Default is Actual.") + String getValueType(); + + void setValueType(String valueType); + + @Validation.Required + @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) + String getReferenceName(); + + void setReferenceName(String referenceName); + + @Validation.Required + @Description("Path to output .txt file.") + String getOutputTxtFilePath(); + + void setOutputTxtFilePath(String outputTxtFilePath); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java new file mode 100644 index 000000000000..00cbcd244768 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java @@ -0,0 +1,103 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.common.Constants; +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; + +/** + * The {@link CdapZendeskOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ +public interface CdapZendeskOptions extends PipelineOptions { + + @Validation.Required + @Description("Zendesk base url.") + String getZendeskBaseUrl(); + + void setZendeskBaseUrl(String zendeskBaseUrl); + + @Validation.Required + @Description("Zendesk admin email.") + String getAdminEmail(); + + void setAdminEmail(String adminEmail); + + @Validation.Required + @Description("Zendesk api token.") + String getApiToken(); + + void setApiToken(String apiToken); + + @Default.String("api/v2") + @Description("Zendesk subdomains.") + String getSubdomains(); + + void setSubdomains(String subdomains); + + @Default.Integer(10000) + @Description("Zendesk maxRetryCount.") + Integer getMaxRetryCount(); + + void setMaxRetryCount(Integer maxRetryCount); + + @Default.Integer(10000) + @Description("Zendesk maxRetryWait.") + Integer getMaxRetryWait(); + + void setMaxRetryWait(Integer maxRetryWait); + + @Default.Integer(10000) + @Description("Zendesk maxRetryJitterWait.") + Integer getMaxRetryJitterWait(); + + void setMaxRetryJitterWait(Integer maxRetryJitterWait); + + @Default.Integer(10) + @Description("Zendesk connectTimeout.") + Integer getConnectTimeout(); + + void setConnectTimeout(Integer connectTimeout); + + @Default.Integer(10) + @Description("Zendesk readTimeout.") + Integer getReadTimeout(); + + void setReadTimeout(Integer readTimeout); + + @Validation.Required + @Description("Zendesk objectsToPull.") + String getObjectsToPull(); + + void setObjectsToPull(String objectsToPull); + + @Validation.Required + @Description("Path to output .txt file.") + String getOutputTxtFilePath(); + + void setOutputTxtFilePath(String outputTxtFilePath); + + @Validation.Required + @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) + String getReferenceName(); + + void setReferenceName(String referenceName); +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java new file mode 100644 index 000000000000..21c76a890018 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/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. + */ + +/** Cdap Plugins template. */ +package org.apache.beam.examples.complete.cdap.options; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java new file mode 100644 index 000000000000..bfb80e02d683 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/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. + */ + +/** Cdap Plugins template. */ +package org.apache.beam.examples.complete.cdap; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java new file mode 100644 index 000000000000..2f77b388ae44 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -0,0 +1,151 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.transforms; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import com.google.gson.JsonElement; +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.data.schema.Schema; +import io.cdap.plugin.hubspot.common.SourceHubspotConfig; +import io.cdap.plugin.hubspot.source.batch.HubspotBatchSource; +import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; +import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; +import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; +import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; +import io.cdap.plugin.servicenow.source.ServiceNowSource; +import io.cdap.plugin.servicenow.source.ServiceNowSourceConfig; +import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSource; +import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSourceConfig; +import java.util.LinkedHashMap; +import java.util.Map; +import org.apache.beam.sdk.io.cdap.CdapIO; +import org.apache.beam.sdk.io.cdap.ConfigWrapper; +import org.apache.hadoop.io.NullWritable; + +/** Different input transformations over the processed data in the pipeline. */ +public class FormatInputTransform { + + /** + * Configures Cdap Zendesk Read transform. + * + * @param pluginConfigParams Cdap Zendesk plugin config parameters + * @return configured Read transform + */ + public static CdapIO.Read readFromCdapZendesk( + Map pluginConfigParams) { + + final ZendeskBatchSourceConfig pluginConfig = + new ConfigWrapper<>(ZendeskBatchSourceConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + return CdapIO.read() + .withCdapPluginClass(ZendeskBatchSource.class) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(StructuredRecord.class); + } + + /** + * Configures Cdap Hubspot Read transform. + * + * @param pluginConfigParams Cdap Hubspot plugin config parameters + * @return configured Read transform + */ + public static CdapIO.Read readFromCdapHubspot( + Map pluginConfigParams) { + + final SourceHubspotConfig pluginConfig = + new ConfigWrapper<>(SourceHubspotConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + return CdapIO.read() + .withCdapPluginClass(HubspotBatchSource.class) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(JsonElement.class); + } + + /** + * Configures Cdap Hubspot Streaming Read transform. + * + * @param pluginConfigParams Cdap Hubspot plugin config parameters + * @return configured Read transform + */ + public static CdapIO.Read readFromCdapHubspotStreaming( + Map pluginConfigParams) { + + final HubspotStreamingSourceConfig pluginConfig = + new ConfigWrapper<>(HubspotStreamingSourceConfig.class) + .withParams(pluginConfigParams) + .build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + return CdapIO.read() + .withCdapPluginClass(HubspotStreamingSource.class) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(String.class); + } + + /** + * Configures Cdap ServiceNow Read transform. + * + * @param pluginConfigParams Cdap ServiceNow plugin config parameters + * @return configured Read transform + */ + public static CdapIO.Read readFromCdapServiceNow( + Map pluginConfigParams) { + + final ServiceNowSourceConfig pluginConfig = + new ConfigWrapper<>(ServiceNowSourceConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + return CdapIO.read() + .withCdapPluginClass(ServiceNowSource.class) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(StructuredRecord.class); + } + + /** + * Configures Cdap Salesforce Read transform. + * + * @param pluginConfigParams Cdap Salesforce plugin config parameters + * @return configured Read transform + */ + @SuppressWarnings("rawtypes") + public static CdapIO.Read readFromCdapSalesforce( + Map pluginConfigParams) { + + final SalesforceSourceConfig pluginConfig = + new ConfigWrapper<>(SalesforceSourceConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + return CdapIO.read() + .withCdapPluginClass(SalesforceBatchSource.class) + .withPluginConfig(pluginConfig) + .withKeyClass(Schema.class) + .withValueClass(LinkedHashMap.class); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java new file mode 100644 index 000000000000..0c87b88da8ee --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.transforms; + +import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; + +import io.cdap.cdap.api.plugin.PluginConfig; +import io.cdap.cdap.etl.api.batch.BatchSink; +import io.cdap.plugin.hubspot.sink.batch.HubspotBatchSink; +import io.cdap.plugin.hubspot.sink.batch.SinkHubspotConfig; +import io.cdap.plugin.salesforce.plugin.sink.batch.CSVRecord; +import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceBatchSink; +import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceSinkConfig; +import java.util.Map; +import org.apache.beam.sdk.io.cdap.CdapIO; +import org.apache.beam.sdk.io.cdap.ConfigWrapper; +import org.apache.hadoop.io.NullWritable; + +/** Different output transformations over the processed data in the pipeline. */ +public class FormatOutputTransform { + + /** + * Configures Cdap Hubspot Write transform. + * + * @param pluginConfigParams Cdap Hubspot plugin config parameters + * @return configured Write transform to Cdap Hubspot + */ + public static CdapIO.Write writeToCdapHubspot( + Map pluginConfigParams, String locksDirPath) { + final PluginConfig pluginConfig = + new ConfigWrapper<>(SinkHubspotConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + Class> pluginClass = HubspotBatchSink.class; + + return CdapIO.write() + .withCdapPluginClass(pluginClass) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(String.class) + .withLocksDirPath(locksDirPath); + } + + /** + * Configures Cdap Salesforce Write transform. + * + * @param pluginConfigParams Cdap Salesforce plugin config parameters + * @return configured Write transform to Cdap Salesforce + */ + public static CdapIO.Write writeToCdapSalesforce( + Map pluginConfigParams, String locksDirPath) { + final PluginConfig pluginConfig = + new ConfigWrapper<>(SalesforceSinkConfig.class).withParams(pluginConfigParams).build(); + + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + Class> pluginClass = SalesforceBatchSink.class; + + return CdapIO.write() + .withCdapPluginClass(pluginClass) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(CSVRecord.class) + .withLocksDirPath(locksDirPath); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java new file mode 100644 index 000000000000..aba98e16fe48 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/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. + */ + +/** Cdap Plugins template. */ +package org.apache.beam.examples.complete.cdap.transforms; diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java new file mode 100644 index 000000000000..432ab77b5545 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.utils; + +import com.google.gson.Gson; +import io.cdap.plugin.salesforce.plugin.sink.batch.CSVRecord; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; + +/** Custom coder for {@link CSVRecord}. */ +public class CsvRecordCoder extends CustomCoder { + + private static final Gson GSON = new Gson(); + private static final CsvRecordCoder CODER = new CsvRecordCoder(); + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + + public static CsvRecordCoder of() { + return CODER; + } + + @Override + public void encode(CSVRecord value, OutputStream outStream) throws IOException { + STRING_CODER.encode(GSON.toJson(value), outStream); + } + + @Override + public CSVRecord decode(InputStream inStream) throws IOException { + return GSON.fromJson(STRING_CODER.decode(inStream), CSVRecord.class); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java new file mode 100644 index 000000000000..1f74406d8703 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.utils; + +import com.google.gson.JsonElement; +import com.google.gson.JsonParser; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import org.apache.beam.sdk.coders.CustomCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.checkerframework.checker.nullness.qual.NonNull; + +/** Custom {@link org.apache.beam.sdk.coders.Coder} for {@link JsonElement}. */ +public class JsonElementCoder extends CustomCoder { + private static final JsonElementCoder CODER = new JsonElementCoder(); + private static final StringUtf8Coder STRING_CODER = StringUtf8Coder.of(); + + public static JsonElementCoder of() { + return CODER; + } + + @Override + public void encode(JsonElement value, @NonNull OutputStream outStream) throws IOException { + if (value != null) { + STRING_CODER.encode(value.toString(), outStream); + } + } + + @Override + public JsonElement decode(@NonNull InputStream inStream) throws IOException { + return JsonParser.parseString(STRING_CODER.decode(inStream)); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java new file mode 100644 index 000000000000..603c612adc10 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.utils; + +import io.cdap.plugin.common.Constants; +import io.cdap.plugin.hubspot.common.BaseHubspotConfig; +import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; +import io.cdap.plugin.hubspot.source.streaming.PullFrequency; +import io.cdap.plugin.salesforce.SalesforceConstants; +import io.cdap.plugin.salesforce.plugin.sink.batch.ErrorHandling; +import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceSinkConfig; +import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; +import io.cdap.plugin.servicenow.source.util.ServiceNowConstants; +import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSourceConfig; +import io.cdap.plugin.zendesk.source.common.config.BaseZendeskSourceConfig; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSinkOptions; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSourceOptions; +import org.apache.beam.examples.complete.cdap.options.CdapServiceNowOptions; +import org.apache.beam.examples.complete.cdap.options.CdapZendeskOptions; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; + +/** + * Class for converting CDAP {@link org.apache.beam.sdk.options.PipelineOptions} to map for {@link + * org.apache.beam.sdk.io.cdap.ConfigWrapper}. + */ +public class PluginConfigOptionsConverter { + + public static Map hubspotOptionsToParamsMap( + CdapHubspotOptions options, boolean isStreaming) { + String apiServerUrl = options.getApiServerUrl(); + ImmutableMap.Builder builder = + ImmutableMap.builder() + .put( + BaseHubspotConfig.API_SERVER_URL, + apiServerUrl != null ? apiServerUrl : BaseHubspotConfig.DEFAULT_API_SERVER_URL) + .put(BaseHubspotConfig.API_KEY, options.getApiKey()) + .put(BaseHubspotConfig.OBJECT_TYPE, options.getObjectType()) + .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()); + if (isStreaming) { + builder.put(HubspotStreamingSourceConfig.PULL_FREQUENCY, PullFrequency.MINUTES_15.getName()); + } + return builder.build(); + } + + public static Map serviceNowOptionsToParamsMap(CdapServiceNowOptions options) { + return ImmutableMap.builder() + .put(ServiceNowConstants.PROPERTY_CLIENT_ID, options.getClientId()) + .put(ServiceNowConstants.PROPERTY_CLIENT_SECRET, options.getClientSecret()) + .put(ServiceNowConstants.PROPERTY_USER, options.getUser()) + .put(ServiceNowConstants.PROPERTY_PASSWORD, options.getPassword()) + .put(ServiceNowConstants.PROPERTY_API_ENDPOINT, options.getRestApiEndpoint()) + .put(ServiceNowConstants.PROPERTY_QUERY_MODE, options.getQueryMode()) + .put(ServiceNowConstants.PROPERTY_TABLE_NAME, options.getTableName()) + .put(ServiceNowConstants.PROPERTY_VALUE_TYPE, options.getValueType()) + .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) + .build(); + } + + public static Map salesforceSourceOptionsToParamsMap( + CdapSalesforceSourceOptions options) { + return ImmutableMap.builder() + .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) + .put(SalesforceConstants.PROPERTY_USERNAME, options.getUsername()) + .put(SalesforceConstants.PROPERTY_PASSWORD, options.getPassword()) + .put(SalesforceConstants.PROPERTY_SECURITY_TOKEN, options.getSecurityToken()) + .put(SalesforceConstants.PROPERTY_CONSUMER_KEY, options.getConsumerKey()) + .put(SalesforceConstants.PROPERTY_CONSUMER_SECRET, options.getConsumerSecret()) + .put(SalesforceConstants.PROPERTY_LOGIN_URL, options.getLoginUrl()) + .put(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME, options.getSObjectName()) + .build(); + } + + public static Map salesforceSinkOptionsToParamsMap( + CdapSalesforceSinkOptions options) { + return ImmutableMap.builder() + .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) + .put(SalesforceConstants.PROPERTY_USERNAME, options.getUsername()) + .put(SalesforceConstants.PROPERTY_PASSWORD, options.getPassword()) + .put(SalesforceConstants.PROPERTY_SECURITY_TOKEN, options.getSecurityToken()) + .put(SalesforceConstants.PROPERTY_CONSUMER_KEY, options.getConsumerKey()) + .put(SalesforceConstants.PROPERTY_CONSUMER_SECRET, options.getConsumerSecret()) + .put(SalesforceConstants.PROPERTY_LOGIN_URL, options.getLoginUrl()) + .put(SalesforceSinkConfig.PROPERTY_SOBJECT, options.getsObject()) + .put(SalesforceSinkConfig.PROPERTY_OPERATION, options.getOperation()) + .put( + SalesforceSinkConfig.PROPERTY_ERROR_HANDLING, + ErrorHandling.valueOf(options.getErrorHandling()).getValue()) + .put(SalesforceSinkConfig.PROPERTY_MAX_BYTES_PER_BATCH, options.getMaxBytesPerBatch()) + .put(SalesforceSinkConfig.PROPERTY_MAX_RECORDS_PER_BATCH, options.getMaxRecordsPerBatch()) + .build(); + } + + public static Map zendeskOptionsToParamsMap(CdapZendeskOptions zendeskOptions) { + return ImmutableMap.builder() + .put(Constants.Reference.REFERENCE_NAME, zendeskOptions.getReferenceName()) + .put(BaseZendeskSourceConfig.PROPERTY_ADMIN_EMAIL, zendeskOptions.getAdminEmail()) + .put(BaseZendeskSourceConfig.PROPERTY_API_TOKEN, zendeskOptions.getApiToken()) + .put(ZendeskBatchSourceConfig.PROPERTY_URL, zendeskOptions.getZendeskBaseUrl()) + .put(ZendeskBatchSourceConfig.PROPERTY_SUBDOMAINS, zendeskOptions.getSubdomains()) + .put(ZendeskBatchSourceConfig.PROPERTY_MAX_RETRY_COUNT, zendeskOptions.getMaxRetryCount()) + .put(ZendeskBatchSourceConfig.PROPERTY_MAX_RETRY_WAIT, zendeskOptions.getMaxRetryWait()) + .put( + ZendeskBatchSourceConfig.PROPERTY_MAX_RETRY_JITTER_WAIT, + zendeskOptions.getMaxRetryJitterWait()) + .put(ZendeskBatchSourceConfig.PROPERTY_CONNECT_TIMEOUT, zendeskOptions.getConnectTimeout()) + .put(ZendeskBatchSourceConfig.PROPERTY_READ_TIMEOUT, zendeskOptions.getReadTimeout()) + .put(BaseZendeskSourceConfig.PROPERTY_OBJECTS_TO_PULL, zendeskOptions.getObjectsToPull()) + .build(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java new file mode 100644 index 000000000000..6cb8df6a8a95 --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.utils; + +import io.cdap.cdap.api.data.format.StructuredRecord; +import io.cdap.cdap.api.data.schema.Schema; +import java.util.List; + +/** + * Class for converting {@link io.cdap.cdap.api.data.format.StructuredRecord} to human-readable + * format. + */ +public class StructuredRecordUtils { + + public static String structuredRecordToString(StructuredRecord structuredRecord) { + if (structuredRecord == null) { + return "{}"; + } + Schema schema = structuredRecord.getSchema(); + StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("{"); + List fields = schema.getFields(); + if (fields != null) { + for (int i = 0; i < fields.size(); i++) { + Schema.Field field = fields.get(i); + Object value = structuredRecord.get(field.getName()); + if (value != null) { + stringBuilder.append("\"").append(field.getName()).append("\": "); + if (String.class.equals(value.getClass())) { + stringBuilder.append("\""); + } + stringBuilder.append(value); + if (String.class.equals(value.getClass())) { + stringBuilder.append("\""); + } + if (i != fields.size() - 1) { + stringBuilder.append(","); + } + } + } + } + stringBuilder.append("}"); + return stringBuilder.toString(); + } +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java new file mode 100644 index 000000000000..db3e73fe7dff --- /dev/null +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/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. + */ + +/** Cdap Plugins template. */ +package org.apache.beam.examples.complete.cdap.utils; diff --git a/examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt b/examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt new file mode 100644 index 000000000000..dfeba37109d0 --- /dev/null +++ b/examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt @@ -0,0 +1,20 @@ +{"properties":[{"name":"name","value":"Company 1 name"},{"name":"description","value":"Company 1 description"}]} +{"properties":[{"name":"name","value":"Company 2 name"},{"name":"description","value":"Company 2 description"}]} +{"properties":[{"name":"name","value":"Company 3 name"},{"name":"description","value":"Company 3 description"}]} +{"properties":[{"name":"name","value":"Company 4 name"},{"name":"description","value":"Company 4 description"}]} +{"properties":[{"name":"name","value":"Company 5 name"},{"name":"description","value":"Company 5 description"}]} +{"properties":[{"name":"name","value":"Company 6 name"},{"name":"description","value":"Company 6 description"}]} +{"properties":[{"name":"name","value":"Company 7 name"},{"name":"description","value":"Company 7 description"}]} +{"properties":[{"name":"name","value":"Company 8 name"},{"name":"description","value":"Company 8 description"}]} +{"properties":[{"name":"name","value":"Company 9 name"},{"name":"description","value":"Company 9 description"}]} +{"properties":[{"name":"name","value":"Company 10 name"},{"name":"description","value":"Company 10 description"}]} +{"properties":[{"name":"name","value":"Company 11 name"},{"name":"description","value":"Company 11 description"}]} +{"properties":[{"name":"name","value":"Company 12 name"},{"name":"description","value":"Company 12 description"}]} +{"properties":[{"name":"name","value":"Company 13 name"},{"name":"description","value":"Company 13 description"}]} +{"properties":[{"name":"name","value":"Company 14 name"},{"name":"description","value":"Company 14 description"}]} +{"properties":[{"name":"name","value":"Company 15 name"},{"name":"description","value":"Company 15 description"}]} +{"properties":[{"name":"name","value":"Company 16 name"},{"name":"description","value":"Company 16 description"}]} +{"properties":[{"name":"name","value":"Company 17 name"},{"name":"description","value":"Company 17 description"}]} +{"properties":[{"name":"name","value":"Company 18 name"},{"name":"description","value":"Company 18 description"}]} +{"properties":[{"name":"name","value":"Company 19 name"},{"name":"description","value":"Company 19 description"}]} +{"properties":[{"name":"name","value":"Company 20 name"},{"name":"description","value":"Company 20 description"}]} \ No newline at end of file diff --git a/examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt b/examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt new file mode 100644 index 000000000000..30951f86d4db --- /dev/null +++ b/examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt @@ -0,0 +1,20 @@ +{"columnNames":["Name"],"values":["Name 1"]} +{"columnNames":["Name"],"values":["Name 2"]} +{"columnNames":["Name"],"values":["Name 3"]} +{"columnNames":["Name"],"values":["Name 4"]} +{"columnNames":["Name"],"values":["Name 5"]} +{"columnNames":["Name"],"values":["Name 6"]} +{"columnNames":["Name"],"values":["Name 7"]} +{"columnNames":["Name"],"values":["Name 8"]} +{"columnNames":["Name"],"values":["Name 9"]} +{"columnNames":["Name"],"values":["Name 10"]} +{"columnNames":["Name"],"values":["Name 11"]} +{"columnNames":["Name"],"values":["Name 12"]} +{"columnNames":["Name"],"values":["Name 13"]} +{"columnNames":["Name"],"values":["Name 14"]} +{"columnNames":["Name"],"values":["Name 15"]} +{"columnNames":["Name"],"values":["Name 16"]} +{"columnNames":["Name"],"values":["Name 17"]} +{"columnNames":["Name"],"values":["Name 18"]} +{"columnNames":["Name"],"values":["Name 19"]} +{"columnNames":["Name"],"values":["Name 20"]} \ No newline at end of file diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 52edc91078a4..0ff82a2b417f 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -71,8 +71,8 @@ dependencies { testImplementation library.java.junit testImplementation library.java.mockito_core testImplementation library.java.testcontainers_postgresql - testImplementation project(":sdks:java:io:hadoop-common") - testImplementation project(":sdks:java:io:hadoop-format") + testImplementation project(path: ":sdks:java:io:hadoop-common", configuration: "testRuntimeMigration") + testImplementation project(path: ":sdks:java:io:hadoop-format", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testImplementation project(path: ":runners:direct-java", configuration: "shadow") testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java index 463cc501a982..36e5e35be074 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java @@ -30,6 +30,8 @@ import io.cdap.plugin.hubspot.source.batch.HubspotInputFormatProvider; import io.cdap.plugin.hubspot.source.streaming.HubspotReceiver; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; +import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceBatchSink; +import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceOutputFormat; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormat; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceInputFormatProvider; @@ -79,6 +81,9 @@ static Plugin getPluginByClass(Class pluginClass) { } else if (pluginClass.equals(HubspotBatchSink.class)) { return Plugin.createBatch( pluginClass, HubspotOutputFormat.class, SourceInputFormatProvider.class); + } else if (pluginClass.equals(SalesforceBatchSink.class)) { + return Plugin.createBatch( + pluginClass, SalesforceOutputFormat.class, SalesforceInputFormatProvider.class); } else if (pluginClass.equals(ServiceNowSource.class)) { return Plugin.createBatch( pluginClass, ServiceNowInputFormat.class, SourceInputFormatProvider.class); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java index 6da476b56f3e..b787f1cd2cb3 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java @@ -98,7 +98,7 @@ public void prepareRun() { cdapPluginObj.prepareRun(getContext()); } catch (Exception e) { LOG.error("Error while prepareRun", e); - throw new IllegalStateException("Error while prepareRun"); + throw new IllegalStateException("Error while prepareRun", e); } if (getPluginType().equals(PluginConstants.PluginType.SOURCE)) { for (Map.Entry entry : diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java index 51018894daf0..c37a9d821bc1 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/BatchContextImpl.java @@ -38,6 +38,9 @@ import io.cdap.cdap.etl.api.lineage.field.FieldOperation; import java.net.URL; import java.sql.Timestamp; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; import java.util.List; import java.util.Map; import javax.annotation.Nullable; @@ -46,6 +49,9 @@ @SuppressWarnings({"TypeParameterUnusedInFormals", "nullness"}) public abstract class BatchContextImpl implements BatchContext { + public static final String DEFAULT_SCHEMA_FIELD_NAME = "Name"; + public static final String DEFAULT_SCHEMA_RECORD_NAME = "Record"; + private final FailureCollectorWrapper failureCollector = new FailureCollectorWrapper(); /** @@ -60,6 +66,12 @@ public abstract class BatchContextImpl implements BatchContext { */ protected OutputFormatProvider outputFormatProvider; + /** + * This should be set after {@link SubmitterLifecycle#prepareRun(Object)} call with passing this + * context object as a param. + */ + protected Map settableArguments = new HashMap<>(); + private final Timestamp startTime = new Timestamp(System.currentTimeMillis()); public InputFormatProvider getInputFormatProvider() { @@ -115,10 +127,11 @@ public T newPluginInstance(String pluginId) throws InstantiationException { return null; } - @Nullable @Override public Schema getInputSchema() { - return null; + List fields = new ArrayList<>(); + fields.add(Schema.Field.of(DEFAULT_SCHEMA_FIELD_NAME, Schema.of(Schema.Type.STRING))); + return Schema.recordOf(DEFAULT_SCHEMA_RECORD_NAME, fields); } @Override @@ -127,8 +140,10 @@ public Schema getInputSchema() { } @Override - public @Nullable Schema getOutputSchema() { - return null; + public Schema getOutputSchema() { + List fields = new ArrayList<>(); + fields.add(Schema.Field.of(DEFAULT_SCHEMA_FIELD_NAME, Schema.of(Schema.Type.STRING))); + return Schema.recordOf(DEFAULT_SCHEMA_RECORD_NAME, fields); } @Override @@ -147,7 +162,33 @@ public boolean datasetExists(String datasetName) throws DatasetManagementExcepti @Override public SettableArguments getArguments() { - return null; + return new SettableArguments() { + @Override + public boolean has(String name) { + return settableArguments.containsKey(name); + } + + @Nullable + @Override + public String get(String name) { + return settableArguments.get(name); + } + + @Override + public void set(String name, String value) { + settableArguments.put(name, value); + } + + @Override + public Map asMap() { + return settableArguments; + } + + @Override + public Iterator> iterator() { + return settableArguments.entrySet().iterator(); + } + }; } @Override diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java index d697909d02ef..0325bc996e2b 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapper.java @@ -18,9 +18,12 @@ package org.apache.beam.sdk.io.cdap.context; import io.cdap.cdap.etl.api.FailureCollector; +import io.cdap.cdap.etl.api.validation.CauseAttributes; import io.cdap.cdap.etl.api.validation.ValidationException; import io.cdap.cdap.etl.api.validation.ValidationFailure; import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; import javax.annotation.Nullable; /** Class FailureCollectorWrapper is a class for collecting ValidationFailure. */ @@ -41,6 +44,21 @@ public ValidationFailure addFailure(String message, @Nullable String correctiveA @Override public ValidationException getOrThrowException() throws ValidationException { + + // We skip schema field validation errors because they are CDAP oriented and don't affect + // anything in our case + for (Iterator failureIterator = failuresCollection.iterator(); + failureIterator.hasNext(); ) { + List causes = failureIterator.next().getCauses(); + if (causes != null) { + for (ValidationFailure.Cause cause : causes) { + String inputField = cause.getAttribute(CauseAttributes.INPUT_SCHEMA_FIELD); + if (BatchContextImpl.DEFAULT_SCHEMA_FIELD_NAME.equals(inputField)) { + failureIterator.remove(); + } + } + } + } if (failuresCollection.isEmpty()) { return new ValidationException(this.failuresCollection); } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java index 0e35c8a06a59..5031cb7e0af7 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/context/FailureCollectorWrapperTest.java @@ -20,6 +20,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertThrows; +import io.cdap.cdap.etl.api.validation.CauseAttributes; import io.cdap.cdap.etl.api.validation.ValidationException; import io.cdap.cdap.etl.api.validation.ValidationFailure; import java.util.ArrayList; @@ -90,4 +91,27 @@ public void getValidationFailures() { assertEquals(errorMessage, exceptionCollector.get(0).getMessage()); assertEquals(0, emptyExceptionCollector.size()); } + + /** + * Should skip schema field validation errors because they are CDAP oriented and don't affect + * anything in our case. + */ + @Test + public void shouldNotThrowForSchemaFieldValidation() { + /** arrange */ + FailureCollectorWrapper failureCollectorWrapper = new FailureCollectorWrapper(); + + /** act */ + RuntimeException error = new RuntimeException("An error with cause has occurred"); + failureCollectorWrapper.addFailure(error.getMessage(), null); + ValidationFailure failure = failureCollectorWrapper.getValidationFailures().get(0); + ValidationFailure.Cause cause = new ValidationFailure.Cause(); + cause.addAttribute( + CauseAttributes.INPUT_SCHEMA_FIELD, BatchContextImpl.DEFAULT_SCHEMA_FIELD_NAME); + failure.getCauses().add(cause); + + /** assert */ + failureCollectorWrapper.getOrThrowException(); + assertEquals(0, failureCollectorWrapper.getValidationFailures().size()); + } } diff --git a/sdks/java/io/hadoop-format/build.gradle b/sdks/java/io/hadoop-format/build.gradle index ec70824a5ab8..2a920de60fa5 100644 --- a/sdks/java/io/hadoop-format/build.gradle +++ b/sdks/java/io/hadoop-format/build.gradle @@ -72,8 +72,6 @@ dependencies { testImplementation project(path: ":sdks:java:io:common", configuration: "testRuntimeMigration") testImplementation project(path: ":sdks:java:testing:test-utils", configuration: "testRuntimeMigration") testImplementation project(":sdks:java:io:jdbc") - testImplementation project(path: ":examples:java", configuration: "testRuntimeMigration") - testImplementation project(path: ":examples:java:twitter", configuration: "testRuntimeMigration") testImplementation "org.elasticsearch.plugin:transport-netty4-client:$elastic_search_version" testImplementation library.java.testcontainers_elasticsearch testImplementation "org.elasticsearch.client:elasticsearch-rest-high-level-client:$elastic_search_version" diff --git a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java index 89a5c5bcd772..b6cb6e627b6f 100644 --- a/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java +++ b/sdks/java/io/hadoop-format/src/test/java/org/apache/beam/sdk/io/hadoop/format/HadoopFormatIOSequenceFileTest.java @@ -29,10 +29,13 @@ import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; -import org.apache.beam.examples.WordCount; import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.metrics.Counter; +import org.apache.beam.sdk.metrics.Distribution; +import org.apache.beam.sdk.metrics.Metrics; import org.apache.beam.sdk.testing.TestPipeline; import org.apache.beam.sdk.testing.TestStream; +import org.apache.beam.sdk.transforms.Count; import org.apache.beam.sdk.transforms.Create; import org.apache.beam.sdk.transforms.DoFn; import org.apache.beam.sdk.transforms.PTransform; @@ -93,6 +96,14 @@ public class HadoopFormatIOSequenceFileTest { (KV element) -> KV.of(new Text(element.getKey()), new LongWritable(element.getValue())); + /** + * \p{L} denotes the category of Unicode letters, so this pattern will match on everything that is + * not a letter. + * + *

It is used for tokenizing strings in the wordcount examples. + */ + private static final String TOKENIZER_PATTERN = "[^\\p{L}]+"; + private static Map computeWordCounts(List sentences) { return sentences.stream() .flatMap(s -> Stream.of(s.split("\\W+"))) @@ -100,6 +111,50 @@ private static Map computeWordCounts(List sentences) { .collect(Collectors.toMap(Function.identity(), s -> 1L, Long::sum)); } + /** + * A PTransform that converts a PCollection containing lines of text into a PCollection of + * formatted word counts. + */ + private static class CountWords + extends PTransform, PCollection>> { + @Override + public PCollection> expand(PCollection lines) { + + // Convert lines of text into individual words. + PCollection words = lines.apply(ParDo.of(new ExtractWordsFn())); + + // Count the number of times each word occurs. + return words.apply(Count.perElement()); + } + } + + /** + * This DoFn tokenizes lines of text into individual words; we pass it to a ParDo in the pipeline. + */ + private static class ExtractWordsFn extends DoFn { + private final Counter emptyLines = Metrics.counter(ExtractWordsFn.class, "emptyLines"); + private final Distribution lineLenDist = + Metrics.distribution(ExtractWordsFn.class, "lineLenDistro"); + + @ProcessElement + public void processElement(@Element String element, OutputReceiver receiver) { + lineLenDist.update(element.length()); + if (element.trim().isEmpty()) { + emptyLines.inc(); + } + + // Split the line into words. + String[] words = element.split(TOKENIZER_PATTERN, -1); + + // Output each word encountered into the output PCollection. + for (String word : words) { + if (!word.isEmpty()) { + receiver.output(word); + } + } + } + } + @Rule public TemporaryFolder tmpFolder = new TemporaryFolder(); @Rule public TestPipeline pipeline = TestPipeline.create(); @@ -158,7 +213,7 @@ private void executeBatchTest(HadoopFormatIO.Write write, St pipeline .apply(Create.of(SENTENCES)) .apply(ParDo.of(new ConvertToLowerCaseFn())) - .apply(new WordCount.CountWords()) + .apply(new CountWords()) .apply( "ConvertToHadoopFormat", ParDo.of(new ConvertToHadoopFormatFn<>(KV_STR_INT_2_TXT_LONGWRITABLE))) @@ -268,7 +323,7 @@ public void streamTest() { .apply(stringsStream) .apply(Window.into(FixedWindows.of(WINDOW_DURATION))) .apply(ParDo.of(new ConvertToLowerCaseFn())) - .apply(new WordCount.CountWords()) + .apply(new CountWords()) .apply( "ConvertToHadoopFormat", ParDo.of(new ConvertToHadoopFormatFn<>(KV_STR_INT_2_TXT_LONGWRITABLE))) From 936ad4cbb75719cef184cf3a717151bb28fab96d Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 26 Oct 2022 17:00:38 +0400 Subject: [PATCH 02/21] Use Akvelon Hubspot plugin dependency --- examples/java/build.gradle | 4 +++- .../main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java | 2 +- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 82c3792f7c40..5cb24f4dffdb 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -69,7 +69,9 @@ dependencies { implementation library.java.cdap_api_commons implementation library.java.cdap_etl_api implementation library.java.cdap_hydrator_common - implementation library.java.cdap_plugin_hubspot + //TODO: modify to 'implementation library.java.cdap_plugin_hubspot', + // when new release with HasOffset interface will be published + implementation "io.github.akvelon:cdap-hubspot-plugins:1.0.2" implementation library.java.cdap_plugin_service_now implementation library.java.cdap_plugin_zendesk implementation library.java.cdap_plugin_salesforce diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java index 36e5e35be074..20cf1c3755df 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java @@ -134,7 +134,7 @@ private static SerializableFunction getOffsetFnForHubspot() { checkArgumentNotNull(json, "Can not get JSON from Hubspot input string"); Object id = json.get(HUBSPOT_ID_FIELD); checkArgumentNotNull(id, "Can not get ID from Hubspot input string"); - return ((Integer) id).longValue(); + return ((Double) id).longValue(); } catch (Exception e) { LOG.error("Can not get offset from json", e); } From 20a1048deb3381aa8a54d308c27bc30dfb90bcc3 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Thu, 3 Nov 2022 14:04:52 +0400 Subject: [PATCH 03/21] Update Hubspot plugin dependency --- examples/java/build.gradle | 2 +- .../complete/cdap/CdapHubspotStreamingToTxt.java | 11 ++++------- 2 files changed, 5 insertions(+), 8 deletions(-) diff --git a/examples/java/build.gradle b/examples/java/build.gradle index 5cb24f4dffdb..ff9aefae7709 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -71,7 +71,7 @@ dependencies { implementation library.java.cdap_hydrator_common //TODO: modify to 'implementation library.java.cdap_plugin_hubspot', // when new release with HasOffset interface will be published - implementation "io.github.akvelon:cdap-hubspot-plugins:1.0.2" + implementation "com.akvelon:cdap-hubspot-plugins:1.0.3" implementation library.java.cdap_plugin_service_now implementation library.java.cdap_plugin_zendesk implementation library.java.cdap_plugin_salesforce diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index 682ac8bc2888..2c7abd9ba790 100644 --- a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -38,7 +38,6 @@ import org.apache.beam.sdk.transforms.windowing.Window; import org.apache.beam.sdk.values.KV; import org.apache.hadoop.io.NullWritable; -import org.joda.time.Duration; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -88,7 +87,6 @@ public class CdapHubspotStreamingToTxt { /* Logger for class.*/ private static final Logger LOG = LoggerFactory.getLogger(CdapHubspotStreamingToTxt.class); - public static final int SECONDS_TO_READ = 30; /** * Main entry point for pipeline execution. @@ -132,13 +130,12 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) .apply( "globalwindow", Window.>into(new GlobalWindows()) - .triggering( - Repeatedly.forever( - AfterProcessingTime.pastFirstElementInPane() - .plusDelayOf(Duration.standardSeconds(SECONDS_TO_READ)))) + .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane())) .discardingFiredPanes()) .apply(Values.create()) - .apply("writeToTxt", TextIO.write().withWindowedWrites().to(options.getTxtFilePath())); + .apply( + "writeToTxt", + TextIO.write().withWindowedWrites().withNumShards(1).to(options.getTxtFilePath())); return pipeline.run(); } From 5124e3f5ab9fdeca862fdb7deef49900d293b434 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Fri, 4 Nov 2022 17:53:28 +0400 Subject: [PATCH 04/21] Move Cdap examples to separate module --- build.gradle.kts | 1 + examples/java/build.gradle | 22 +--- examples/java/cdap/build.gradle | 115 ++++++++++++++++++ .../cdap/CdapHubspotStreamingToTxt.java | 0 .../complete/cdap/CdapHubspotToTxt.java | 0 .../complete/cdap/CdapSalesforceToTxt.java | 0 .../complete/cdap/CdapServiceNowToTxt.java | 0 .../complete/cdap/CdapZendeskToTxt.java | 0 .../beam/examples/complete/cdap/README.md | 0 .../complete/cdap/TxtToCdapHubspot.java | 0 .../complete/cdap/TxtToCdapSalesforce.java | 0 .../cdap/options/CdapHubspotOptions.java | 0 .../cdap/options/CdapSalesforceOptions.java | 0 .../options/CdapSalesforceSinkOptions.java | 0 .../options/CdapSalesforceSourceOptions.java | 0 .../cdap/options/CdapServiceNowOptions.java | 0 .../cdap/options/CdapZendeskOptions.java | 0 .../complete/cdap/options/package-info.java | 0 .../examples/complete/cdap/package-info.java | 0 .../cdap/transforms/FormatInputTransform.java | 0 .../transforms/FormatOutputTransform.java | 0 .../cdap/transforms/package-info.java | 0 .../complete/cdap/utils/CsvRecordCoder.java | 0 .../complete/cdap/utils/JsonElementCoder.java | 0 .../utils/PluginConfigOptionsConverter.java | 0 .../cdap/utils/StructuredRecordUtils.java | 0 .../complete/cdap/utils/package-info.java | 0 settings.gradle.kts | 1 + 28 files changed, 118 insertions(+), 21 deletions(-) create mode 100644 examples/java/cdap/build.gradle rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/README.md (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java (100%) rename examples/java/{ => cdap}/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java (100%) diff --git a/build.gradle.kts b/build.gradle.kts index 2556db7bc9ae..9fca0c63562f 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -201,6 +201,7 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:core:buildNeeded") dependsOn(":sdks:java:core:buildDependents") dependsOn(":examples:java:preCommit") + dependsOn(":examples:java:cdap:preCommit") dependsOn(":examples:java:twitter:preCommit") dependsOn(":sdks:java:extensions:sql:jdbc:preCommit") dependsOn(":sdks:java:javadoc:allJavadoc") diff --git a/examples/java/build.gradle b/examples/java/build.gradle index ff9aefae7709..aa51dcfeae85 100644 --- a/examples/java/build.gradle +++ b/examples/java/build.gradle @@ -60,21 +60,9 @@ dependencies { implementation project(":sdks:java:extensions:python") implementation project(":sdks:java:io:google-cloud-platform") implementation project(":sdks:java:io:kafka") - implementation project(":sdks:java:io:cdap") - implementation project(":sdks:java:io:hadoop-common") implementation project(":sdks:java:extensions:ml") implementation library.java.avro implementation library.java.bigdataoss_util - implementation library.java.cdap_api - implementation library.java.cdap_api_commons - implementation library.java.cdap_etl_api - implementation library.java.cdap_hydrator_common - //TODO: modify to 'implementation library.java.cdap_plugin_hubspot', - // when new release with HasOffset interface will be published - implementation "com.akvelon:cdap-hubspot-plugins:1.0.3" - implementation library.java.cdap_plugin_service_now - implementation library.java.cdap_plugin_zendesk - implementation library.java.cdap_plugin_salesforce implementation library.java.google_api_client implementation library.java.google_api_services_bigquery implementation library.java.google_api_services_pubsub @@ -84,7 +72,6 @@ dependencies { implementation library.java.google_code_gson implementation library.java.google_http_client implementation library.java.google_oauth_client - implementation library.java.hadoop_common implementation library.java.jackson_databind implementation library.java.joda_time implementation library.java.protobuf_java @@ -177,11 +164,4 @@ task wordCount(type:JavaExec) { classpath = sourceSets.main.runtimeClasspath systemProperties = System.getProperties() args = ["--output=/tmp/ouput.txt"] -} - -task executeCdap (type:JavaExec) { - mainClass = System.getProperty("mainClass") - classpath = sourceSets.main.runtimeClasspath - systemProperties System.getProperties() - args System.getProperty("exec.args", "").split() -} +} \ No newline at end of file diff --git a/examples/java/cdap/build.gradle b/examples/java/cdap/build.gradle new file mode 100644 index 000000000000..610ea573cb91 --- /dev/null +++ b/examples/java/cdap/build.gradle @@ -0,0 +1,115 @@ +/* + * 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. + */ + +import groovy.json.JsonOutput + +plugins { + id 'java' + id 'org.apache.beam.module' + id 'com.github.johnrengelman.shadow' +} + +applyJavaNature( + exportJavadoc: false, + automaticModuleName: 'org.apache.beam.examples.complete.cdap', +) +provideIntegrationTestingDependencies() +enableJavaPerformanceTesting() + +description = "Apache Beam :: Examples :: Java :: CDAP " +ext.summary = """Apache Beam SDK provides a simple, Java-based +interface for processing virtually any size data. This +artifact includes CDAP Apache Beam Java SDK examples.""" + +/** Define the list of runners which execute a precommit test. + * Some runners are run from separate projects, see the preCommit task below + * for details. + */ +def preCommitRunners = ["directRunner", "flinkRunner"] +for (String runner : preCommitRunners) { + configurations.create(runner + "PreCommit") +} + +dependencies { + implementation enforcedPlatform(library.java.google_cloud_platform_libraries_bom) + implementation project(path: ":sdks:java:core", configuration: "shadow") + implementation project(":sdks:java:io:cdap") + implementation project(":sdks:java:io:hadoop-common") + implementation library.java.cdap_api + implementation library.java.cdap_api_commons + implementation library.java.cdap_etl_api + implementation library.java.cdap_hydrator_common + //TODO: modify to 'implementation library.java.cdap_plugin_hubspot', + // when new release with HasOffset interface will be published + implementation "com.akvelon:cdap-hubspot-plugins:1.0.3" + implementation library.java.cdap_plugin_service_now + implementation library.java.cdap_plugin_zendesk + implementation library.java.cdap_plugin_salesforce + implementation library.java.google_code_gson + implementation library.java.hadoop_common + implementation library.java.slf4j_api + implementation library.java.vendored_guava_26_0_jre + runtimeOnly project(path: ":runners:direct-java", configuration: "shadow") + testImplementation project(path: ":runners:direct-java", configuration: "shadow") + testImplementation library.java.hamcrest + testImplementation library.java.junit + + // Add dependencies for the PreCommit configurations + // For each runner a project level dependency on the examples project. + for (String runner : preCommitRunners) { + delegate.add(runner + "PreCommit", project(":examples:java:cdap")) + delegate.add(runner + "PreCommit", project(path: ":examples:java:cdap", configuration: "testRuntimeMigration")) + } + directRunnerPreCommit project(path: ":runners:direct-java", configuration: "shadow") + flinkRunnerPreCommit project(":runners:flink:${project.ext.latestFlinkVersion}") +} + +/* + * Create a ${runner}PreCommit task for each runner which runs a set + * of integration tests for WordCount and WindowedWordCount. + */ +def preCommitRunnerClass = [ + directRunner: "org.apache.beam.runners.direct.DirectRunner", + flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", +] + +for (String runner : preCommitRunners) { + tasks.create(name: runner + "PreCommit", type: Test) { + def preCommitBeamTestPipelineOptions = [ + "--runner=" + preCommitRunnerClass[runner], + ] + classpath = configurations."${runner}PreCommit" + forkEvery 1 + maxParallelForks 4 + systemProperty "beamTestPipelineOptions", JsonOutput.toJson(preCommitBeamTestPipelineOptions) + } +} + +/* Define a common precommit task which depends on all the individual precommits. */ +task preCommit() { + for (String runner : preCommitRunners) { + dependsOn runner + "PreCommit" + } +} + +task executeCdap (type:JavaExec) { + mainClass = System.getProperty("mainClass") + classpath = sourceSets.main.runtimeClasspath + systemProperties System.getProperties() + args System.getProperty("exec.args", "").split() +} diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/README.md rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatOutputTransform.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/package-info.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/CsvRecordCoder.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/JsonElementCoder.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java diff --git a/examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java similarity index 100% rename from examples/java/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java rename to examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/package-info.java diff --git a/settings.gradle.kts b/settings.gradle.kts index 2ddc12fb36bd..ab7178a086df 100644 --- a/settings.gradle.kts +++ b/settings.gradle.kts @@ -52,6 +52,7 @@ include(":release:go-licenses:py") include(":examples:java") include(":examples:java:twitter") +include(":examples:java:cdap") include(":examples:kotlin") include(":examples:multi-language") include(":model:fn-execution") From f4accbeff1b54a6b58ac42d3524bc18ecbd1808d Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Fri, 4 Nov 2022 19:01:50 +0400 Subject: [PATCH 05/21] Fix Hubspot Streaming example --- .../cdap/transforms/FormatInputTransform.java | 12 +++- .../complete/cdap/utils/GetOffsetUtils.java | 58 +++++++++++++++++++ sdks/java/io/cdap/build.gradle | 1 - .../org/apache/beam/sdk/io/cdap/CdapIO.java | 5 +- .../apache/beam/sdk/io/cdap/MappingUtils.java | 56 +++--------------- .../apache/beam/sdk/io/cdap/CdapIOTest.java | 5 +- 6 files changed, 79 insertions(+), 58 deletions(-) create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index 2f77b388ae44..733a1b22a08a 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -24,6 +24,7 @@ import io.cdap.cdap.api.data.schema.Schema; import io.cdap.plugin.hubspot.common.SourceHubspotConfig; import io.cdap.plugin.hubspot.source.batch.HubspotBatchSource; +import io.cdap.plugin.hubspot.source.streaming.HubspotReceiver; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; @@ -34,8 +35,11 @@ import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSourceConfig; import java.util.LinkedHashMap; import java.util.Map; +import org.apache.beam.examples.complete.cdap.utils.GetOffsetUtils; import org.apache.beam.sdk.io.cdap.CdapIO; import org.apache.beam.sdk.io.cdap.ConfigWrapper; +import org.apache.beam.sdk.io.cdap.MappingUtils; +import org.apache.beam.sdk.io.cdap.Plugin; import org.apache.hadoop.io.NullWritable; /** Different input transformations over the processed data in the pipeline. */ @@ -96,11 +100,15 @@ public static CdapIO.Read readFromCdapHubspotStreaming( new ConfigWrapper<>(HubspotStreamingSourceConfig.class) .withParams(pluginConfigParams) .build(); - checkStateNotNull(pluginConfig, "Plugin config can't be null."); + MappingUtils.registerStreamingPlugin( + HubspotStreamingSource.class, + GetOffsetUtils.getOffsetFnForHubspot(), + HubspotReceiver.class); + return CdapIO.read() - .withCdapPluginClass(HubspotStreamingSource.class) + .withCdapPlugin(Plugin.createStreaming(HubspotStreamingSource.class)) .withPluginConfig(pluginConfig) .withKeyClass(NullWritable.class) .withValueClass(String.class); diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java new file mode 100644 index 000000000000..a03f723694f9 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.utils; + +import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; + +import com.google.gson.Gson; +import java.util.HashMap; +import org.apache.beam.sdk.io.cdap.Plugin; +import org.apache.beam.sdk.transforms.SerializableFunction; +import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.reflect.TypeToken; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Class for getting a {@link SerializableFunction} that defines how to get record offset for + * different CDAP {@link Plugin} classes. + */ +public class GetOffsetUtils { + + private static final Logger LOG = LoggerFactory.getLogger(GetOffsetUtils.class); + + private static final String HUBSPOT_ID_FIELD = "vid"; + private static final Gson GSON = new Gson(); + + public static SerializableFunction getOffsetFnForHubspot() { + return input -> { + if (input != null) { + try { + HashMap json = + GSON.fromJson(input, new TypeToken>() {}.getType()); + checkArgumentNotNull(json, "Can not get JSON from Hubspot input string"); + Object id = json.get(HUBSPOT_ID_FIELD); + checkArgumentNotNull(id, "Can not get ID from Hubspot input string"); + return ((Double) id).longValue(); + } catch (Exception e) { + LOG.error("Can not get offset from json", e); + } + } + return 0L; + }; + } +} diff --git a/sdks/java/io/cdap/build.gradle b/sdks/java/io/cdap/build.gradle index 0ff82a2b417f..11b1966eeb28 100644 --- a/sdks/java/io/cdap/build.gradle +++ b/sdks/java/io/cdap/build.gradle @@ -52,7 +52,6 @@ dependencies { implementation library.java.cdap_plugin_zendesk implementation library.java.commons_lang3 implementation library.java.guava - implementation library.java.google_code_gson implementation library.java.hadoop_common implementation library.java.hadoop_mapreduce_client_core implementation library.java.jackson_core diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java index 5590bb061654..c54d5d502ba6 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java @@ -278,10 +278,9 @@ public PCollection> expand(PBegin input) { if (cdapPlugin.isUnbounded()) { SparkReceiverIO.Read reader = SparkReceiverIO.read() - .withGetOffsetFn(getOffsetFnForPluginClass(cdapPlugin.getPluginClass(), valueClass)) + .withGetOffsetFn(getOffsetFnForPluginClass(cdapPlugin.getPluginClass())) .withSparkReceiverBuilder( - getReceiverBuilderByPluginClass( - cdapPlugin.getPluginClass(), pluginConfig, valueClass)); + getReceiverBuilderByPluginClass(cdapPlugin.getPluginClass(), pluginConfig)); try { Coder coder = input.getPipeline().getCoderRegistry().getCoder(valueClass); PCollection values = input.apply(reader).setCoder(coder); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java index 20cf1c3755df..29c8b8d469ff 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java @@ -17,10 +17,8 @@ */ package org.apache.beam.sdk.io.cdap; -import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import com.google.gson.Gson; import io.cdap.cdap.api.plugin.PluginConfig; import io.cdap.plugin.common.SourceInputFormatProvider; import io.cdap.plugin.hubspot.sink.batch.HubspotBatchSink; @@ -28,8 +26,6 @@ import io.cdap.plugin.hubspot.source.batch.HubspotBatchSource; import io.cdap.plugin.hubspot.source.batch.HubspotInputFormat; import io.cdap.plugin.hubspot.source.batch.HubspotInputFormatProvider; -import io.cdap.plugin.hubspot.source.streaming.HubspotReceiver; -import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceBatchSink; import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceOutputFormat; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; @@ -44,22 +40,15 @@ import java.util.Map; import org.apache.beam.sdk.io.sparkreceiver.ReceiverBuilder; import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.reflect.TypeToken; import org.apache.commons.lang3.tuple.ImmutablePair; import org.apache.commons.lang3.tuple.Pair; import org.apache.spark.streaming.receiver.Receiver; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; /** Util class for mapping plugins. */ public class MappingUtils { - private static final Logger LOG = LoggerFactory.getLogger(MappingUtils.class); - private static final String HUBSPOT_ID_FIELD = "vid"; - private static final Gson GSON = new Gson(); - private static final Map< - Class, Pair, ReceiverBuilder>>> + Class, Pair, Class>>> REGISTERED_PLUGINS; static { @@ -87,8 +76,6 @@ static Plugin getPluginByClass(Class pluginClass) { } else if (pluginClass.equals(ServiceNowSource.class)) { return Plugin.createBatch( pluginClass, ServiceNowInputFormat.class, SourceInputFormatProvider.class); - } else if (pluginClass.equals(HubspotStreamingSource.class)) { - return Plugin.createStreaming(pluginClass); } throw new UnsupportedOperationException( String.format("Given plugin class '%s' is not supported!", pluginClass.getName())); @@ -97,18 +84,13 @@ static Plugin getPluginByClass(Class pluginClass) { /** Gets a {@link ReceiverBuilder} by CDAP {@link Plugin} class. */ @SuppressWarnings("unchecked") static ReceiverBuilder> getReceiverBuilderByPluginClass( - Class pluginClass, PluginConfig pluginConfig, Class valueClass) { + Class pluginClass, PluginConfig pluginConfig) { checkArgument(pluginClass != null, "Plugin class can not be null!"); checkArgument(pluginConfig != null, "Plugin config can not be null!"); - checkArgument(valueClass != null, "Value class can not be null!"); - if (pluginClass.equals(HubspotStreamingSource.class) && String.class.equals(valueClass)) { - ReceiverBuilder> receiverBuilder = - new ReceiverBuilder<>(HubspotReceiver.class).withConstructorArgs(pluginConfig); - return (ReceiverBuilder>) receiverBuilder; - } if (REGISTERED_PLUGINS.containsKey(pluginClass)) { - return (ReceiverBuilder>) - REGISTERED_PLUGINS.get(pluginClass).getRight(); + Class> receiverClass = + (Class>) REGISTERED_PLUGINS.get(pluginClass).getRight(); + return new ReceiverBuilder<>(receiverClass).withConstructorArgs(pluginConfig); } throw new UnsupportedOperationException( String.format("Given plugin class '%s' is not supported!", pluginClass.getName())); @@ -121,26 +103,8 @@ static Plugin getPluginByClass(Class pluginClass) { public static void registerStreamingPlugin( Class pluginClass, SerializableFunction getOffsetFn, - ReceiverBuilder> receiverBuilder) { - REGISTERED_PLUGINS.put(pluginClass, new ImmutablePair<>(getOffsetFn, receiverBuilder)); - } - - private static SerializableFunction getOffsetFnForHubspot() { - return input -> { - if (input != null) { - try { - HashMap json = - GSON.fromJson(input, new TypeToken>() {}.getType()); - checkArgumentNotNull(json, "Can not get JSON from Hubspot input string"); - Object id = json.get(HUBSPOT_ID_FIELD); - checkArgumentNotNull(id, "Can not get ID from Hubspot input string"); - return ((Double) id).longValue(); - } catch (Exception e) { - LOG.error("Can not get offset from json", e); - } - } - return 0L; - }; + Class> receiverClass) { + REGISTERED_PLUGINS.put(pluginClass, new ImmutablePair<>(getOffsetFn, receiverClass)); } /** @@ -148,11 +112,7 @@ private static SerializableFunction getOffsetFnForHubspot() { * Plugin} class. */ @SuppressWarnings("unchecked") - static SerializableFunction getOffsetFnForPluginClass( - Class pluginClass, Class valueClass) { - if (pluginClass.equals(HubspotStreamingSource.class) && String.class.equals(valueClass)) { - return (SerializableFunction) getOffsetFnForHubspot(); - } + static SerializableFunction getOffsetFnForPluginClass(Class pluginClass) { if (REGISTERED_PLUGINS.containsKey(pluginClass)) { return (SerializableFunction) REGISTERED_PLUGINS.get(pluginClass).getLeft(); } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java index e18126e69acf..d22761368eca 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java @@ -43,7 +43,6 @@ import org.apache.beam.sdk.io.cdap.context.BatchSourceContextImpl; import org.apache.beam.sdk.io.cdap.streaming.EmployeeReceiver; import org.apache.beam.sdk.io.cdap.streaming.EmployeeStreamingSource; -import org.apache.beam.sdk.io.sparkreceiver.ReceiverBuilder; import org.apache.beam.sdk.options.PipelineOptionsFactory; import org.apache.beam.sdk.testing.PAssert; import org.apache.beam.sdk.testing.TestPipeline; @@ -185,9 +184,7 @@ public void testReadFromCdapStreamingPlugin() { EmployeeConfig pluginConfig = new ConfigWrapper<>(EmployeeConfig.class).withParams(TEST_EMPLOYEE_PARAMS_MAP).build(); MappingUtils.registerStreamingPlugin( - EmployeeStreamingSource.class, - Long::valueOf, - new ReceiverBuilder<>(EmployeeReceiver.class).withConstructorArgs(pluginConfig)); + EmployeeStreamingSource.class, Long::valueOf, EmployeeReceiver.class); CdapIO.Read read = CdapIO.read() From d9fed6342ee96efb8c96231097e279487b0ad344 Mon Sep 17 00:00:00 2001 From: Alex Kosolapov Date: Sat, 5 Nov 2022 16:38:28 -0700 Subject: [PATCH 06/21] Updates to CdapIO examples readme --- .../beam/examples/complete/cdap/README.md | 36 ++++++++++++++++--- 1 file changed, 31 insertions(+), 5 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 09ba15207dd3..c86d3d8ba514 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -15,14 +15,12 @@ under the License. --> -# Apache Beam pipeline examples to ingest data from CDAP plugin to TXT file +# Apache Beam pipeline examples for CdapIO and CDAP plugins -This directory contains set of [Apache Beam](https://beam.apache.org/) pipeline examples that create a pipeline to read data -from a [CDAP](https://cdap.atlassian.net/wiki/spaces/DOCS/overview?homepageId=379748484) plugin -and write data into .txt file (and vice versa). +This directory contains set of [Apache Beam](https://beam.apache.org/) pipeline examples to read data +from a [CDAP plugin](https://github.com/data-integrations) and write data into .txt file (and vice versa). Supported CDAP plugins: - - [Hubspot](https://github.com/data-integrations/hubspot) - [Salesforce](https://github.com/data-integrations/salesforce) - [ServiceNow](https://github.com/data-integrations/servicenow-plugins) @@ -50,6 +48,13 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap -Dexec.args="--= --=" ``` +`CdapHubspotToTxt` pipeline parameters: +- `apikey` - Hubspot OAuth2 API Key +- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) +- `txtFilePath` - output path. + +Please see CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) for more information. + To execute this pipeline, specify the parameters in the following format: ```bash @@ -68,6 +73,13 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap -Dexec.args="--= --=" ``` +`CdapHubspotStreamingToTxt` pipeline parameters: +- `apikey` - Hubspot OAuth2 API Key +- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) +- `txtFilePath` - output path. + +Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. + To execute this pipeline, specify the parameters in the following format: ```bash @@ -86,6 +98,14 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtT -Dexec.args="--= --=" ``` +`TxtToCdapHubspot` pipeline parameters: +- `apikey` - Hubspot OAuth2 API Key +- `objectType` - Hubspot objects to pull supported by [Hubspot Streaming Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) +- `txtFilePath` - input file path +- `locksDirPath` + +Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. + To execute this pipeline, specify the parameters in the following format: ```bash @@ -120,6 +140,8 @@ To execute this pipeline, specify the parameters in the following format: --outputTxtFilePath=your-path-to-file ``` +Please see CDAP [ServiceNow Batch Source](https://github.com/data-integrations/servicenow-plugins/blob/develop/docs/ServiceNow-batchsource.md) for more information. + ## Running the CdapSalesforceToTxt pipeline example Gradle 'executeCdap' task allows to run the pipeline via the following command: @@ -143,6 +165,8 @@ To execute this pipeline, specify the parameters in the following format: --txtFilePath=your-path-to-file ``` +Please see CDAP [Salesforce Batch Source](https://github.com/data-integrations/servicenow-plugins/blob/develop/docs/ServiceNow-batchsource.md) for more information. + ## Running the TxtToCdapSalesforce pipeline example Gradle 'executeCdap' task allows to run the pipeline via the following command: @@ -170,6 +194,7 @@ To execute this pipeline, specify the parameters in the following format: --maxBytesPerBatch=9999999 \ --locksDirPath=your-path ``` +Please see CDAP [Salesforce Batch Sink](https://github.com/data-integrations/salesforce/blob/develop/docs/Salesforce-batchsink.md) for more information. ## Running the CdapZendeskToTxt pipeline example @@ -195,3 +220,4 @@ To execute this pipeline, specify the parameters in the following format: --objectsToPull=your-objects-to-pull (example: Groups) \ --outputTxtFilePath=your-path-to-file ``` +Please see CDAP [Zendesk Batch Source](https://github.com/data-integrations/zendesk/blob/develop/docs/Zendesk-batchsource.md) for more information. From 1e086233144719af02a72ebc1328482d0605dc2a Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Mon, 7 Nov 2022 12:10:03 +0400 Subject: [PATCH 07/21] Add description for locksDirPath pipeline parameter --- build.gradle.kts | 1 - .../java/org/apache/beam/examples/complete/cdap/README.md | 2 +- .../examples/complete/cdap/options/CdapHubspotOptions.java | 5 ++++- .../complete/cdap/options/CdapSalesforceSinkOptions.java | 5 ++++- 4 files changed, 9 insertions(+), 4 deletions(-) diff --git a/build.gradle.kts b/build.gradle.kts index 9fca0c63562f..2556db7bc9ae 100644 --- a/build.gradle.kts +++ b/build.gradle.kts @@ -201,7 +201,6 @@ tasks.register("javaPreCommit") { dependsOn(":sdks:java:core:buildNeeded") dependsOn(":sdks:java:core:buildDependents") dependsOn(":examples:java:preCommit") - dependsOn(":examples:java:cdap:preCommit") dependsOn(":examples:java:twitter:preCommit") dependsOn(":sdks:java:extensions:sql:jdbc:preCommit") dependsOn(":sdks:java:javadoc:allJavadoc") diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index c86d3d8ba514..76261c7c60fa 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -102,7 +102,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtT - `apikey` - Hubspot OAuth2 API Key - `objectType` - Hubspot objects to pull supported by [Hubspot Streaming Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) - `txtFilePath` - input file path -- `locksDirPath` +- `locksDirPath` - locks directory path where locks will be stored. This parameter is needed for Hadoop External Synchronization (mechanism for acquiring locks related to the write job). Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java index b8628dd0ad01..9923bef28168 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java @@ -57,7 +57,10 @@ public interface CdapHubspotOptions extends PipelineOptions { void setTxtFilePath(String txtFilePath); - @Description("Locks directory path where locks will be stored.") + @Description( + "Locks directory path where locks will be stored." + + "This parameter is needed for Hadoop External Synchronization" + + "(mechanism for acquiring locks related to the write job).") String getLocksDirPath(); void setLocksDirPath(String locksDirPath); diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java index 16af4621898d..ba1bd8f0fcc8 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java @@ -64,7 +64,10 @@ public interface CdapSalesforceSinkOptions extends CdapSalesforceOptions { void setsObject(String sObject); - @Description("Locks directory path where locks will be stored.") + @Description( + "Locks directory path where locks will be stored." + + "This parameter is needed for Hadoop External Synchronization" + + "(mechanism for acquiring locks related to the write job).") String getLocksDirPath(); void setLocksDirPath(String locksDirPath); From b5490e47d2d54abe2d49e8bf484eacc1f8852533 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Mon, 7 Nov 2022 17:28:37 +0400 Subject: [PATCH 08/21] Refactoring --- .../cdap/CdapHubspotStreamingToTxt.java | 15 ++++--- .../complete/cdap/CdapHubspotToTxt.java | 12 ++--- .../complete/cdap/CdapSalesforceToTxt.java | 4 +- .../complete/cdap/CdapServiceNowToTxt.java | 4 +- .../complete/cdap/CdapZendeskToTxt.java | 4 +- .../beam/examples/complete/cdap/README.md | 20 ++++----- .../complete/cdap/TxtToCdapHubspot.java | 12 ++--- .../complete/cdap/TxtToCdapSalesforce.java | 4 +- .../cdap/options/CdapHubspotOptions.java | 14 ------ .../cdap/options/CdapHubspotSinkOptions.java | 44 +++++++++++++++++++ .../options/CdapHubspotSourceOptions.java | 38 ++++++++++++++++ .../cdap/options/CdapSalesforceOptions.java | 10 ++--- .../options/CdapSalesforceSinkOptions.java | 12 +++++ .../options/CdapSalesforceSourceOptions.java | 13 ++++++ .../cdap/options/CdapServiceNowOptions.java | 11 +++-- .../cdap/options/CdapZendeskOptions.java | 11 +++-- .../utils/PluginConfigOptionsConverter.java | 2 + 17 files changed, 166 insertions(+), 64 deletions(-) create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSinkOptions.java create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index 2c7abd9ba790..58b0f12c2742 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -19,7 +19,7 @@ import com.google.gson.JsonElement; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotSourceOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; @@ -74,7 +74,7 @@ * --apikey=your-api-key \ * --referenceName=your-reference-name \ * --objectType=Contacts \ - * --txtFilePath=your-path-to-output-file + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -94,8 +94,8 @@ public class CdapHubspotStreamingToTxt { * @param args Command line arguments to the pipeline. */ public static void main(String[] args) { - CdapHubspotOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + CdapHubspotSourceOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotSourceOptions.class); // Create the pipeline Pipeline pipeline = Pipeline.create(options); @@ -107,7 +107,7 @@ public static void main(String[] args) { * * @param options arguments to the pipeline */ - public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + public static PipelineResult run(Pipeline pipeline, CdapHubspotSourceOptions options) { Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, true); LOG.info("Starting Cdap-Hubspot-streaming-to-txt pipeline with parameters: {}", paramsMap); @@ -135,7 +135,10 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) .apply(Values.create()) .apply( "writeToTxt", - TextIO.write().withWindowedWrites().withNumShards(1).to(options.getTxtFilePath())); + TextIO.write() + .withWindowedWrites() + .withNumShards(1) + .to(options.getOutputTxtFilePathPrefix())); return pipeline.run(); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java index dc7df5f3c87b..3f29adf50005 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java @@ -19,7 +19,7 @@ import com.google.gson.JsonElement; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotSourceOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; @@ -71,7 +71,7 @@ * --apikey=your-api-key \ * --referenceName=your-reference-name \ * --objectType=Contacts \ - * --txtFilePath=your-path-to-output-file + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -91,8 +91,8 @@ public class CdapHubspotToTxt { * @param args Command line arguments to the pipeline. */ public static void main(String[] args) { - CdapHubspotOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + CdapHubspotSourceOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotSourceOptions.class); // Create the pipeline Pipeline pipeline = Pipeline.create(options); @@ -104,7 +104,7 @@ public static void main(String[] args) { * * @param options arguments to the pipeline */ - public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + public static PipelineResult run(Pipeline pipeline, CdapHubspotSourceOptions options) { Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); LOG.info("Starting Cdap-Hubspot-to-txt pipeline with parameters: {}", paramsMap); @@ -135,7 +135,7 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) .apply(Values.create()) - .apply("writeToTxt", TextIO.write().to(options.getTxtFilePath())); + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePathPrefix())); return pipeline.run(); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java index 58a5b34a0438..43c6cbb82e62 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java @@ -74,7 +74,7 @@ * --loginUrl=your-login-url \ * --sObjectName=object-name \ * --referenceName=your-reference-name \ - * --txtFilePath=your-path-to-file + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -130,7 +130,7 @@ public static PipelineResult run(Pipeline pipeline, CdapSalesforceSourceOptions .apply(MapValues.into(TypeDescriptors.strings()).via(LinkedHashMap::toString)) .setCoder(KvCoder.of(SerializableCoder.of(Schema.class), StringUtf8Coder.of())) .apply(Values.create()) - .apply("writeToTxt", TextIO.write().to(options.getTxtFilePath())); + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePathPrefix())); return pipeline.run(); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java index 88a7076659a4..fe00dbcaab4d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapServiceNowToTxt.java @@ -78,7 +78,7 @@ * --tableName=your-table \ * --valueType=Actual \ * --referenceName=your-reference-name \ - * --outputTxtFilePath=your-path-to-file + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -136,7 +136,7 @@ public static PipelineResult run(Pipeline pipeline, CdapServiceNowOptions option KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) .apply(Values.create()) - .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePath())); + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePathPrefix())); return pipeline.run(); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java index d01b19534d99..fbeb20839693 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapZendeskToTxt.java @@ -75,7 +75,7 @@ * --apiToken=your-token \ * --objectsToPull=Groups \ * --referenceName=your-reference-name \ - * --outputTxtFilePath=your-path-to-file + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -133,7 +133,7 @@ public static PipelineResult run(Pipeline pipeline, CdapZendeskOptions options) KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) .apply(Values.create()) - .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePath())); + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePathPrefix())); return pipeline.run(); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 76261c7c60fa..7fb96d46282d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -51,7 +51,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap `CdapHubspotToTxt` pipeline parameters: - `apikey` - Hubspot OAuth2 API Key - `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) -- `txtFilePath` - output path. +- `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. Please see CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) for more information. @@ -61,7 +61,7 @@ To execute this pipeline, specify the parameters in the following format: --apikey=your-api-key \ --referenceName=your-reference-name \ --objectType=Contacts \ - --txtFilePath=your-path-to-output-file + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix ``` ## Running the CdapHubspotStreamingToTxt pipeline example @@ -76,7 +76,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap `CdapHubspotStreamingToTxt` pipeline parameters: - `apikey` - Hubspot OAuth2 API Key - `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) -- `txtFilePath` - output path. +- `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. @@ -86,7 +86,7 @@ To execute this pipeline, specify the parameters in the following format: --apikey=your-api-key \ --referenceName=your-reference-name \ --objectType=Contacts \ - --txtFilePath=your-path-to-output-file + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix ``` ## Running the TxtToCdapHubspot pipeline example @@ -101,7 +101,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtT `TxtToCdapHubspot` pipeline parameters: - `apikey` - Hubspot OAuth2 API Key - `objectType` - Hubspot objects to pull supported by [Hubspot Streaming Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) -- `txtFilePath` - input file path +- `inputTxtFilePath` - input .txt file path - `locksDirPath` - locks directory path where locks will be stored. This parameter is needed for Hadoop External Synchronization (mechanism for acquiring locks related to the write job). Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. @@ -112,7 +112,7 @@ To execute this pipeline, specify the parameters in the following format: --apikey=your-api-key \ --referenceName=your-reference-name \ --objectType=your-object-type \ - --txtFilePath=your-path-to-input-file \ + --inputTxtFilePath=your-path-to-input-txt-file \ --locksDirPath=your-path-to-locks-dir ``` @@ -137,7 +137,7 @@ To execute this pipeline, specify the parameters in the following format: --tableName=your-table \ --valueType=Actual \ --referenceName=your-reference-name \ - --outputTxtFilePath=your-path-to-file + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix ``` Please see CDAP [ServiceNow Batch Source](https://github.com/data-integrations/servicenow-plugins/blob/develop/docs/ServiceNow-batchsource.md) for more information. @@ -162,7 +162,7 @@ To execute this pipeline, specify the parameters in the following format: --loginUrl=your-login-url \ --sObjectName=object-name \ --referenceName=your-reference-name \ - --txtFilePath=your-path-to-file + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix ``` Please see CDAP [Salesforce Batch Source](https://github.com/data-integrations/servicenow-plugins/blob/develop/docs/ServiceNow-batchsource.md) for more information. @@ -187,7 +187,7 @@ To execute this pipeline, specify the parameters in the following format: --loginUrl=your-login-url \ --sObject=CustomObject__c \ --referenceName=your-reference-name \ - --txtFilePath=your-path-to-file \ + --inputTxtFilePath=your-path-to-txt-file \ --operation=Insert \ --errorHandling=Stop on error \ --maxRecordsPerBatch=10 \ @@ -218,6 +218,6 @@ To execute this pipeline, specify the parameters in the following format: --connectTimeout=your-connection-timeout \ --readTimeout=your-read-timeout \ --objectsToPull=your-objects-to-pull (example: Groups) \ - --outputTxtFilePath=your-path-to-file + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix ``` Please see CDAP [Zendesk Batch Source](https://github.com/data-integrations/zendesk/blob/develop/docs/Zendesk-batchsource.md) for more information. diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java index 7c26b504d936..2deaa544a465 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java @@ -20,7 +20,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotSinkOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatOutputTransform; import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; import org.apache.beam.sdk.Pipeline; @@ -67,7 +67,7 @@ * --apikey=your-api-key \ * --referenceName=your-reference-name \ * --objectType=your-object-type \ - * --txtFilePath=your-path-to-input-file \ + * --inputTxtFilePath=your-path-to-input-file \ * --locksDirPath=your-path-to-locks-dir * } * @@ -88,8 +88,8 @@ public class TxtToCdapHubspot { * @param args Command line arguments to the pipeline. */ public static void main(String[] args) { - CdapHubspotOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotOptions.class); + CdapHubspotSinkOptions options = + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotSinkOptions.class); checkStateNotNull(options.getLocksDirPath(), "locksDirPath can not be null!"); @@ -103,7 +103,7 @@ public static void main(String[] args) { * * @param options arguments to the pipeline */ - public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) { + public static PipelineResult run(Pipeline pipeline, CdapHubspotSinkOptions options) { Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); LOG.info("Starting Txt-to-Cdap-Hubspot pipeline with parameters: {}", paramsMap); @@ -116,7 +116,7 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotOptions options) */ pipeline - .apply("readFromTxt", TextIO.read().from(options.getTxtFilePath())) + .apply("readFromTxt", TextIO.read().from(options.getInputTxtFilePath())) .apply( MapElements.into(new TypeDescriptor>() {}) .via(json -> KV.of(NullWritable.get(), json))) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java index 1386eb0bf092..337c463cdfc6 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java @@ -78,7 +78,7 @@ * --loginUrl=your-login-url \ * --sObject=CustomObject__c \ * --referenceName=your-reference-name \ - * --txtFilePath=your-path-to-file \ + * --inputTxtFilePath=your-path-to-input-file \ * --maxRecordsPerBatch=10 \ * --maxBytesPerBatch=9999999 \ * --operation=Insert \ @@ -132,7 +132,7 @@ public static PipelineResult run(Pipeline pipeline, CdapSalesforceSinkOptions op */ pipeline - .apply("readFromTxt", TextIO.read().from(options.getTxtFilePath())) + .apply("readFromTxt", TextIO.read().from(options.getInputTxtFilePath())) .apply( MapElements.into(new TypeDescriptor>() {}) .via( diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java index 9923bef28168..aae31a2657bf 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java @@ -50,18 +50,4 @@ public interface CdapHubspotOptions extends PipelineOptions { String getReferenceName(); void setReferenceName(String referenceName); - - @Validation.Required - @Description("Path to input/output .txt file.") - String getTxtFilePath(); - - void setTxtFilePath(String txtFilePath); - - @Description( - "Locks directory path where locks will be stored." - + "This parameter is needed for Hadoop External Synchronization" - + "(mechanism for acquiring locks related to the write job).") - String getLocksDirPath(); - - void setLocksDirPath(String locksDirPath); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSinkOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSinkOptions.java new file mode 100644 index 000000000000..8653a5ac3c43 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSinkOptions.java @@ -0,0 +1,44 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link CdapHubspotSinkOptions} interface provides the custom execution options passed by the + * executor at the command-line for {@link org.apache.beam.examples.complete.cdap.TxtToCdapHubspot} + * example. + */ +public interface CdapHubspotSinkOptions extends CdapHubspotOptions { + + @Validation.Required + @Description("Input .txt file path with Hubspot records.") + String getInputTxtFilePath(); + + void setInputTxtFilePath(String inputTxtFilePath); + + @Validation.Required + @Description( + "Locks directory path where locks will be stored." + + "This parameter is needed for Hadoop External Synchronization" + + "(mechanism for acquiring locks related to the write job).") + String getLocksDirPath(); + + void setLocksDirPath(String locksDirPath); +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java new file mode 100644 index 000000000000..755e80d3aec9 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java @@ -0,0 +1,38 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link CdapHubspotSourceOptions} interface provides the custom execution options passed by + * the executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.CdapHubspotToTxt} and {@link + * org.apache.beam.examples.complete.cdap.CdapHubspotStreamingToTxt} examples. + */ +public interface CdapHubspotSourceOptions extends CdapHubspotOptions { + + @Validation.Required + @Description( + "Output .txt file path with file name prefix." + + "It will write a set of files with names like {prefix}-###.") + String getOutputTxtFilePathPrefix(); + + void setOutputTxtFilePathPrefix(String outputTxtFilePathPrefix); +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java index da83d92629e7..efe612671077 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java @@ -23,6 +23,10 @@ import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; +/** + * The {@link CdapSalesforceOptions} interface provides the custom execution options passed by the + * executor at the command-line. + */ public interface CdapSalesforceOptions extends PipelineOptions { @Validation.Required @@ -66,10 +70,4 @@ public interface CdapSalesforceOptions extends PipelineOptions { String getLoginUrl(); void setLoginUrl(String loginUrl); - - @Validation.Required - @Description("Path to input/output .txt file.") - String getTxtFilePath(); - - void setTxtFilePath(String txtFilePath); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java index ba1bd8f0fcc8..b3f90619e0bd 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSinkOptions.java @@ -20,6 +20,11 @@ import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Validation; +/** + * The {@link CdapSalesforceSinkOptions} interface provides the custom execution options passed by + * the executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.TxtToCdapSalesforce} example. + */ public interface CdapSalesforceSinkOptions extends CdapSalesforceOptions { @Validation.Required @@ -64,6 +69,7 @@ public interface CdapSalesforceSinkOptions extends CdapSalesforceOptions { void setsObject(String sObject); + @Validation.Required @Description( "Locks directory path where locks will be stored." + "This parameter is needed for Hadoop External Synchronization" @@ -71,4 +77,10 @@ public interface CdapSalesforceSinkOptions extends CdapSalesforceOptions { String getLocksDirPath(); void setLocksDirPath(String locksDirPath); + + @Validation.Required + @Description("Input .txt file path with Salesforce records.") + String getInputTxtFilePath(); + + void setInputTxtFilePath(String inputTxtFilePath); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java index 0dc7fe152fea..db5d86ded732 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceSourceOptions.java @@ -21,6 +21,11 @@ import org.apache.beam.sdk.options.Description; import org.apache.beam.sdk.options.Validation; +/** + * The {@link CdapSalesforceSourceOptions} interface provides the custom execution options passed by + * the executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.CdapSalesforceToTxt} example. + */ public interface CdapSalesforceSourceOptions extends CdapSalesforceOptions { @Validation.Required @@ -28,4 +33,12 @@ public interface CdapSalesforceSourceOptions extends CdapSalesforceOptions { String getSObjectName(); void setSObjectName(String sObjectName); + + @Validation.Required + @Description( + "Path to output folder with filename prefix." + + "It will write a set of .txt files with names like {prefix}-###.") + String getOutputTxtFilePathPrefix(); + + void setOutputTxtFilePathPrefix(String outputTxtFilePathPrefix); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java index f197971a502a..b06494d8f14d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java @@ -24,7 +24,8 @@ /** * The {@link CdapServiceNowOptions} interface provides the custom execution options passed by the - * executor at the command-line. + * executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.CdapServiceNowToTxt} example. */ public interface CdapServiceNowOptions extends PipelineOptions { @@ -93,8 +94,10 @@ public interface CdapServiceNowOptions extends PipelineOptions { void setReferenceName(String referenceName); @Validation.Required - @Description("Path to output .txt file.") - String getOutputTxtFilePath(); + @Description( + "Path to output folder with filename prefix." + + "It will write a set of .txt files with names like {prefix}-###.") + String getOutputTxtFilePathPrefix(); - void setOutputTxtFilePath(String outputTxtFilePath); + void setOutputTxtFilePathPrefix(String outputTxtFilePathPrefix); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java index 00cbcd244768..4e9a6bd2e7e7 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java @@ -25,7 +25,8 @@ /** * The {@link CdapZendeskOptions} interface provides the custom execution options passed by the - * executor at the command-line. + * executor at the command-line for {@link org.apache.beam.examples.complete.cdap.CdapZendeskToTxt} + * example. */ public interface CdapZendeskOptions extends PipelineOptions { @@ -90,10 +91,12 @@ public interface CdapZendeskOptions extends PipelineOptions { void setObjectsToPull(String objectsToPull); @Validation.Required - @Description("Path to output .txt file.") - String getOutputTxtFilePath(); + @Description( + "Path to output folder with filename prefix." + + "It will write a set of .txt files with names like {prefix}-###.") + String getOutputTxtFilePathPrefix(); - void setOutputTxtFilePath(String outputTxtFilePath); + void setOutputTxtFilePathPrefix(String outputTxtFilePathPrefix); @Validation.Required @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index 603c612adc10..dccf9c2414fd 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -54,6 +54,8 @@ public static Map hubspotOptionsToParamsMap( .put(BaseHubspotConfig.OBJECT_TYPE, options.getObjectType()) .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()); if (isStreaming) { + // Pull frequency is not implemented in CdapIO, but still needs to be passed for + // HubspotStreamingSource plugin builder.put(HubspotStreamingSourceConfig.PULL_FREQUENCY, PullFrequency.MINUTES_15.getName()); } return builder.build(); From 29ad3740043dacf618e569b93e98f76d0d08da2c Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 8 Nov 2022 16:18:08 +0400 Subject: [PATCH 09/21] Refactor Plugin class --- .../cdap/transforms/FormatInputTransform.java | 12 +-- .../org/apache/beam/sdk/io/cdap/CdapIO.java | 36 +++---- .../apache/beam/sdk/io/cdap/MappingUtils.java | 57 +---------- .../org/apache/beam/sdk/io/cdap/Plugin.java | 95 +++++++++++++++---- .../apache/beam/sdk/io/cdap/CdapIOTest.java | 13 ++- .../apache/beam/sdk/io/cdap/PluginTest.java | 12 +-- 6 files changed, 118 insertions(+), 107 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index 733a1b22a08a..8cd9e646d4fb 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -38,7 +38,6 @@ import org.apache.beam.examples.complete.cdap.utils.GetOffsetUtils; import org.apache.beam.sdk.io.cdap.CdapIO; import org.apache.beam.sdk.io.cdap.ConfigWrapper; -import org.apache.beam.sdk.io.cdap.MappingUtils; import org.apache.beam.sdk.io.cdap.Plugin; import org.apache.hadoop.io.NullWritable; @@ -102,13 +101,12 @@ public static CdapIO.Read readFromCdapHubspotStreaming( .build(); checkStateNotNull(pluginConfig, "Plugin config can't be null."); - MappingUtils.registerStreamingPlugin( - HubspotStreamingSource.class, - GetOffsetUtils.getOffsetFnForHubspot(), - HubspotReceiver.class); - return CdapIO.read() - .withCdapPlugin(Plugin.createStreaming(HubspotStreamingSource.class)) + .withCdapPlugin( + Plugin.createStreaming( + HubspotStreamingSource.class, + GetOffsetUtils.getOffsetFnForHubspot(), + HubspotReceiver.class)) .withPluginConfig(pluginConfig) .withKeyClass(NullWritable.class) .withValueClass(String.class); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java index c54d5d502ba6..ebcdcb971ccc 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java @@ -17,9 +17,7 @@ */ package org.apache.beam.sdk.io.cdap; -import static org.apache.beam.sdk.io.cdap.MappingUtils.getOffsetFnForPluginClass; import static org.apache.beam.sdk.io.cdap.MappingUtils.getPluginByClass; -import static org.apache.beam.sdk.io.cdap.MappingUtils.getReceiverBuilderByPluginClass; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; @@ -32,6 +30,7 @@ import org.apache.beam.sdk.coders.Coder; import org.apache.beam.sdk.io.hadoop.format.HDFSSynchronization; import org.apache.beam.sdk.io.hadoop.format.HadoopFormatIO; +import org.apache.beam.sdk.io.sparkreceiver.ReceiverBuilder; import org.apache.beam.sdk.io.sparkreceiver.SparkReceiverIO; import org.apache.beam.sdk.transforms.MapElements; import org.apache.beam.sdk.transforms.PTransform; @@ -45,6 +44,7 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputFormat; import org.apache.hadoop.mapreduce.OutputFormat; +import org.apache.spark.streaming.receiver.Receiver; import org.checkerframework.checker.nullness.qual.Nullable; /** @@ -150,7 +150,7 @@ * *

{@link Plugin} is the Wrapper class for the Cdap Plugin. It contains main information about * the Plugin. The object of the {@link Plugin} class can be created with the {@link - * Plugin#createStreaming(Class)} method. Method requires {@link + * Plugin#createStreaming(Class, SerializableFunction, Class)} method. Method requires {@link * io.cdap.cdap.etl.api.streaming.StreamingSource} class parameter. * *

Every Cdap Plugin has its {@link PluginConfig} class with necessary fields to configure the @@ -193,7 +193,7 @@ public abstract static class Read extends PTransform getCdapPlugin(); /** * Depending on selected {@link HadoopFormatIO} type ({@link InputFormat} or {@link @@ -219,7 +219,7 @@ abstract static class Builder { abstract Builder setPluginConfig(PluginConfig config); - abstract Builder setCdapPlugin(Plugin plugin); + abstract Builder setCdapPlugin(Plugin plugin); abstract Builder setKeyClass(Class keyClass); @@ -229,7 +229,7 @@ abstract static class Builder { } /** Sets a CDAP {@link Plugin}. */ - public Read withCdapPlugin(Plugin plugin) { + public Read withCdapPlugin(Plugin plugin) { checkArgument(plugin != null, "Cdap plugin can not be null"); return toBuilder().setCdapPlugin(plugin).build(); } @@ -237,7 +237,7 @@ public Read withCdapPlugin(Plugin plugin) { /** Sets a CDAP Plugin class. */ public Read withCdapPluginClass(Class cdapPluginClass) { checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null"); - Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass); + Plugin plugin = MappingUtils.getPluginByClass(cdapPluginClass); return toBuilder().setCdapPlugin(plugin).build(); } @@ -261,7 +261,7 @@ public Read withValueClass(Class valueClass) { @Override public PCollection> expand(PBegin input) { - Plugin cdapPlugin = getCdapPlugin(); + Plugin cdapPlugin = getCdapPlugin(); checkStateNotNull(cdapPlugin, "withCdapPluginClass() is required"); PluginConfig pluginConfig = getPluginConfig(); @@ -276,11 +276,15 @@ public PCollection> expand(PBegin input) { cdapPlugin.withConfig(pluginConfig); if (cdapPlugin.isUnbounded()) { + SerializableFunction getOffsetFn = cdapPlugin.getGetOffsetFn(); + checkStateNotNull(getOffsetFn, "Plugin get offset function can't be null!"); + ReceiverBuilder> receiverBuilder = cdapPlugin.getReceiverBuilder(); + checkStateNotNull(receiverBuilder, "Plugin Receiver builder can't be null!"); + SparkReceiverIO.Read reader = SparkReceiverIO.read() - .withGetOffsetFn(getOffsetFnForPluginClass(cdapPlugin.getPluginClass())) - .withSparkReceiverBuilder( - getReceiverBuilderByPluginClass(cdapPlugin.getPluginClass(), pluginConfig)); + .withGetOffsetFn(getOffsetFn) + .withSparkReceiverBuilder(receiverBuilder); try { Coder coder = input.getPipeline().getCoderRegistry().getCoder(valueClass); PCollection values = input.apply(reader).setCoder(coder); @@ -306,7 +310,7 @@ public abstract static class Write extends PTransform abstract @Nullable PluginConfig getPluginConfig(); - abstract @Nullable Plugin getCdapPlugin(); + abstract @Nullable Plugin getCdapPlugin(); /** * Depending on selected {@link HadoopFormatIO} type ({@link InputFormat} or {@link @@ -340,7 +344,7 @@ abstract static class Builder { abstract Builder setPluginConfig(PluginConfig config); - abstract Builder setCdapPlugin(Plugin plugin); + abstract Builder setCdapPlugin(Plugin plugin); abstract Builder setKeyClass(Class keyClass); @@ -352,7 +356,7 @@ abstract static class Builder { } /** Sets a CDAP {@link Plugin}. */ - public Write withCdapPlugin(Plugin plugin) { + public Write withCdapPlugin(Plugin plugin) { checkArgument(plugin != null, "Cdap plugin can not be null"); return toBuilder().setCdapPlugin(plugin).build(); } @@ -360,7 +364,7 @@ public Write withCdapPlugin(Plugin plugin) { /** Sets a CDAP Plugin class. */ public Write withCdapPluginClass(Class cdapPluginClass) { checkArgument(cdapPluginClass != null, "Cdap plugin class can not be null"); - Plugin plugin = getPluginByClass(cdapPluginClass); + Plugin plugin = getPluginByClass(cdapPluginClass); return toBuilder().setCdapPlugin(plugin).build(); } @@ -390,7 +394,7 @@ public Write withValueClass(Class valueClass) { @Override public PDone expand(PCollection> input) { - Plugin cdapPlugin = getCdapPlugin(); + Plugin cdapPlugin = getCdapPlugin(); checkStateNotNull(cdapPlugin, "withCdapPluginClass() is required"); PluginConfig pluginConfig = getPluginConfig(); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java index 29c8b8d469ff..8ac28a9b6b38 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/MappingUtils.java @@ -19,7 +19,6 @@ import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkArgument; -import io.cdap.cdap.api.plugin.PluginConfig; import io.cdap.plugin.common.SourceInputFormatProvider; import io.cdap.plugin.hubspot.sink.batch.HubspotBatchSink; import io.cdap.plugin.hubspot.sink.batch.HubspotOutputFormat; @@ -36,27 +35,12 @@ import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSource; import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormat; import io.cdap.plugin.zendesk.source.batch.ZendeskInputFormatProvider; -import java.util.HashMap; -import java.util.Map; -import org.apache.beam.sdk.io.sparkreceiver.ReceiverBuilder; -import org.apache.beam.sdk.transforms.SerializableFunction; -import org.apache.commons.lang3.tuple.ImmutablePair; -import org.apache.commons.lang3.tuple.Pair; -import org.apache.spark.streaming.receiver.Receiver; /** Util class for mapping plugins. */ public class MappingUtils { - private static final Map< - Class, Pair, Class>>> - REGISTERED_PLUGINS; - - static { - REGISTERED_PLUGINS = new HashMap<>(); - } - /** Gets a {@link Plugin} by its class. */ - static Plugin getPluginByClass(Class pluginClass) { + static Plugin getPluginByClass(Class pluginClass) { checkArgument(pluginClass != null, "Plugin class can not be null!"); if (pluginClass.equals(SalesforceBatchSource.class)) { return Plugin.createBatch( @@ -80,43 +64,4 @@ static Plugin getPluginByClass(Class pluginClass) { throw new UnsupportedOperationException( String.format("Given plugin class '%s' is not supported!", pluginClass.getName())); } - - /** Gets a {@link ReceiverBuilder} by CDAP {@link Plugin} class. */ - @SuppressWarnings("unchecked") - static ReceiverBuilder> getReceiverBuilderByPluginClass( - Class pluginClass, PluginConfig pluginConfig) { - checkArgument(pluginClass != null, "Plugin class can not be null!"); - checkArgument(pluginConfig != null, "Plugin config can not be null!"); - if (REGISTERED_PLUGINS.containsKey(pluginClass)) { - Class> receiverClass = - (Class>) REGISTERED_PLUGINS.get(pluginClass).getRight(); - return new ReceiverBuilder<>(receiverClass).withConstructorArgs(pluginConfig); - } - throw new UnsupportedOperationException( - String.format("Given plugin class '%s' is not supported!", pluginClass.getName())); - } - - /** - * Register new CDAP Streaming {@link Plugin} class providing corresponding {@param getOffsetFn} - * and {@param receiverBuilder} params. - */ - public static void registerStreamingPlugin( - Class pluginClass, - SerializableFunction getOffsetFn, - Class> receiverClass) { - REGISTERED_PLUGINS.put(pluginClass, new ImmutablePair<>(getOffsetFn, receiverClass)); - } - - /** - * Gets a {@link SerializableFunction} that defines how to get record offset for CDAP {@link - * Plugin} class. - */ - @SuppressWarnings("unchecked") - static SerializableFunction getOffsetFnForPluginClass(Class pluginClass) { - if (REGISTERED_PLUGINS.containsKey(pluginClass)) { - return (SerializableFunction) REGISTERED_PLUGINS.get(pluginClass).getLeft(); - } - throw new UnsupportedOperationException( - String.format("Given plugin class '%s' is not supported!", pluginClass.getName())); - } } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java index b787f1cd2cb3..43992120c84e 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java @@ -18,6 +18,7 @@ package org.apache.beam.sdk.io.cdap; import static org.apache.beam.sdk.util.Preconditions.checkStateNotNull; +import static org.apache.beam.vendor.guava.v26_0_jre.com.google.common.base.Preconditions.checkState; import com.google.auto.value.AutoValue; import io.cdap.cdap.api.plugin.PluginConfig; @@ -39,15 +40,18 @@ import org.apache.beam.sdk.io.cdap.context.BatchSinkContextImpl; import org.apache.beam.sdk.io.cdap.context.BatchSourceContextImpl; import org.apache.beam.sdk.io.cdap.context.StreamingSourceContextImpl; +import org.apache.beam.sdk.io.sparkreceiver.ReceiverBuilder; +import org.apache.beam.sdk.transforms.SerializableFunction; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.MRJobConfig; +import org.apache.spark.streaming.receiver.Receiver; import org.slf4j.Logger; import org.slf4j.LoggerFactory; /** Class wrapper for a CDAP plugin. */ @AutoValue @SuppressWarnings({"rawtypes", "unchecked"}) -public abstract class Plugin { +public abstract class Plugin { private static final Logger LOG = LoggerFactory.getLogger(Plugin.class); private static final String PREPARE_RUN_METHOD_NAME = "prepareRun"; @@ -69,8 +73,19 @@ public abstract class Plugin { /** Gets InputFormatProvider or OutputFormatProvider class for a plugin. */ public @Nullable abstract Class getFormatProviderClass(); + public @Nullable abstract Class> getReceiverClass(); + + /** + * Gets a {@link SerializableFunction} that defines how to get record offset for CDAP {@link + * Plugin} class. + */ + public @Nullable abstract SerializableFunction getGetOffsetFn(); + + public @Nullable abstract SerializableFunction + getGetReceiverArgsFromConfigFn(); + /** Sets a plugin config. */ - public Plugin withConfig(PluginConfig pluginConfig) { + public Plugin withConfig(PluginConfig pluginConfig) { this.pluginConfig = pluginConfig; return this; } @@ -129,7 +144,7 @@ private void instantiateCdapPluginObj() { } /** Sets a plugin Hadoop configuration. */ - public Plugin withHadoopConfiguration(Class formatKeyClass, Class formatValueClass) { + public Plugin withHadoopConfiguration(Class formatKeyClass, Class formatValueClass) { Class formatClass = getFormatClass(); checkStateNotNull(formatClass, "Format class can't be null!"); PluginConstants.Format formatType = getFormatType(); @@ -144,7 +159,7 @@ public Plugin withHadoopConfiguration(Class formatKeyClass, Class formatVa } /** Sets a plugin Hadoop configuration. */ - public Plugin withHadoopConfiguration(Configuration hadoopConfiguration) { + public Plugin withHadoopConfiguration(Configuration hadoopConfiguration) { this.hadoopConfiguration = hadoopConfiguration; return this; } @@ -226,10 +241,30 @@ public Boolean isUnbounded() { return isUnbounded; } + /** Gets a {@link ReceiverBuilder}. */ + public ReceiverBuilder> getReceiverBuilder() { + checkState(isUnbounded(), "Receiver Builder is supported only for unbounded plugins"); + + Class pluginClass = getPluginClass(); + Class> receiverClass = getReceiverClass(); + SerializableFunction getReceiverArgsFromConfigFn = + getGetReceiverArgsFromConfigFn(); + PluginConfig pluginConfig = getPluginConfig(); + + checkStateNotNull(pluginConfig, "Plugin config can not be null!"); + checkStateNotNull(pluginClass, "Plugin class can not be null!"); + checkStateNotNull(receiverClass, "Receiver class can not be null!"); + checkStateNotNull( + getReceiverArgsFromConfigFn, "Get receiver args from config function can not be null!"); + + return new ReceiverBuilder<>(receiverClass) + .withConstructorArgs(getReceiverArgsFromConfigFn.apply(pluginConfig)); + } + /** Creates a batch plugin instance. */ - public static Plugin createBatch( + public static Plugin createBatch( Class newPluginClass, Class newFormatClass, Class newFormatProviderClass) { - return builder() + return Plugin.builder() .setPluginClass(newPluginClass) .setFormatClass(newFormatClass) .setFormatProviderClass(newFormatProviderClass) @@ -239,32 +274,58 @@ public static Plugin createBatch( } /** Creates a streaming plugin instance. */ - public static Plugin createStreaming(Class newPluginClass) { - return builder() + public static Plugin createStreaming( + Class newPluginClass, + SerializableFunction getOffsetFn, + Class> receiverClass, + SerializableFunction getReceiverArgsFromConfigFn) { + return Plugin.builder() .setPluginClass(newPluginClass) .setPluginType(Plugin.initPluginType(newPluginClass)) .setContext(Plugin.initContext(newPluginClass)) + .setGetOffsetFn(getOffsetFn) + .setReceiverClass(receiverClass) + .setGetReceiverArgsFromConfigFn(getReceiverArgsFromConfigFn) .build(); } + /** + * Creates a streaming plugin instance with default function for getting args for {@link + * Receiver}. + */ + public static Plugin createStreaming( + Class newPluginClass, + SerializableFunction getOffsetFn, + Class> receiverClass) { + return createStreaming( + newPluginClass, getOffsetFn, receiverClass, config -> new Object[] {config}); + } + /** Creates a plugin builder instance. */ - public static Builder builder() { - return new AutoValue_Plugin.Builder(); + public static Builder builder() { + return new AutoValue_Plugin.Builder<>(); } /** Builder class for a {@link Plugin}. */ @AutoValue.Builder - public abstract static class Builder { - public abstract Builder setPluginClass(Class newPluginClass); + public abstract static class Builder { + public abstract Builder setPluginClass(Class newPluginClass); + + public abstract Builder setFormatClass(Class newFormatClass); + + public abstract Builder setFormatProviderClass(Class newFormatProviderClass); + + public abstract Builder setGetOffsetFn(SerializableFunction getOffsetFn); - public abstract Builder setFormatClass(Class newFormatClass); + public abstract Builder setGetReceiverArgsFromConfigFn( + SerializableFunction getReceiverArgsFromConfigFn); - public abstract Builder setFormatProviderClass(Class newFormatProviderClass); + public abstract Builder setReceiverClass(Class> receiverClass); - public abstract Builder setPluginType(PluginConstants.PluginType newPluginType); + public abstract Builder setPluginType(PluginConstants.PluginType newPluginType); - public abstract Builder setContext(BatchContextImpl context); + public abstract Builder setContext(BatchContextImpl context); - public abstract Plugin build(); + public abstract Plugin build(); } } diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java index d22761368eca..6fcfdbdfc24d 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java @@ -97,7 +97,7 @@ public void testReadBuildsCorrectly() { .withKeyClass(String.class) .withValueClass(String.class); - Plugin cdapPlugin = read.getCdapPlugin(); + Plugin cdapPlugin = read.getCdapPlugin(); assertNotNull(cdapPlugin); assertEquals(EmployeeBatchSource.class, cdapPlugin.getPluginClass()); assertEquals(EmployeeInputFormat.class, cdapPlugin.getFormatClass()); @@ -183,12 +183,15 @@ public void testReadFromCdapStreamingPlugin() { EmployeeConfig pluginConfig = new ConfigWrapper<>(EmployeeConfig.class).withParams(TEST_EMPLOYEE_PARAMS_MAP).build(); - MappingUtils.registerStreamingPlugin( - EmployeeStreamingSource.class, Long::valueOf, EmployeeReceiver.class); CdapIO.Read read = CdapIO.read() - .withCdapPlugin(Plugin.createStreaming(EmployeeStreamingSource.class)) + .withCdapPlugin( + Plugin.createStreaming( + EmployeeStreamingSource.class, + Long::valueOf, + EmployeeReceiver.class, + config -> new Object[] {config})) .withPluginConfig(pluginConfig) .withKeyClass(String.class) .withValueClass(String.class); @@ -221,7 +224,7 @@ public void testWriteBuildsCorrectly() { .withValueClass(String.class) .withLocksDirPath(tmpFolder.getRoot().getAbsolutePath()); - Plugin cdapPlugin = write.getCdapPlugin(); + Plugin cdapPlugin = write.getCdapPlugin(); assertNotNull(cdapPlugin); assertNotNull(write.getLocksDirPath()); assertEquals(EmployeeBatchSink.class, cdapPlugin.getPluginClass()); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginTest.java index 2fcfe6f36c0b..4881c4573367 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/PluginTest.java @@ -64,8 +64,8 @@ public class PluginTest { @Test public void testBuildingSourcePluginWithCDAPClasses() { try { - Plugin serviceNowSourcePlugin = - Plugin.createBatch( + Plugin serviceNowSourcePlugin = + Plugin.createBatch( ServiceNowSource.class, ServiceNowInputFormat.class, SourceInputFormatProvider.class) @@ -92,8 +92,8 @@ public void testBuildingSourcePluginWithCDAPClasses() { @Test public void testSettingPluginType() { - Plugin serviceNowSourcePlugin = - Plugin.createBatch( + Plugin serviceNowSourcePlugin = + Plugin.createBatch( ServiceNowSource.class, ServiceNowInputFormat.class, SourceInputFormatProvider.class) @@ -107,8 +107,8 @@ public void testSettingPluginType() { @SuppressWarnings("UnusedVariable") public void testSettingPluginTypeFailed() { try { - Plugin serviceNowSourcePlugin = - Plugin.createBatch(Object.class, Object.class, Object.class) + Plugin serviceNowSourcePlugin = + Plugin.createBatch(Object.class, Object.class, Object.class) .withConfig(serviceNowSourceConfig) .withHadoopConfiguration(Schema.class, MapWritable.class); fail("This should have thrown an exception"); From 393714b43c3d14af64a4eac7034a6f646e4abc04 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Tue, 8 Nov 2022 17:12:46 +0400 Subject: [PATCH 10/21] Add javadoc --- .../org/apache/beam/sdk/io/cdap/CdapIO.java | 10 +++++-- .../org/apache/beam/sdk/io/cdap/Plugin.java | 30 +++++++++++++++++-- 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java index ebcdcb971ccc..3da7c4deb749 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java @@ -151,7 +151,9 @@ *

{@link Plugin} is the Wrapper class for the Cdap Plugin. It contains main information about * the Plugin. The object of the {@link Plugin} class can be created with the {@link * Plugin#createStreaming(Class, SerializableFunction, Class)} method. Method requires {@link - * io.cdap.cdap.etl.api.streaming.StreamingSource} class parameter. + * io.cdap.cdap.etl.api.streaming.StreamingSource} class, {@code getOffsetFn} which is a {@link + * SerializableFunction} that defines how to get {@code Long offset} from {@code V record}, Spark + * {@link Receiver} class parameters. * *

Every Cdap Plugin has its {@link PluginConfig} class with necessary fields to configure the * Plugin. You can set the {@link Map} of your parameters with the {@link @@ -169,7 +171,11 @@ * // Read using CDAP streaming plugin * p.apply("ReadStreaming", * CdapIO.read() - * .withCdapPlugin(Plugin.createStreaming(EmployeeStreamingSource.class)) + * .withCdapPlugin( + * Plugin.createStreaming( + * EmployeeStreamingSource.class, + * Long::valueOf, + * EmployeeReceiver.class)) * .withPluginConfig(pluginConfig) * .withKeyClass(String.class) * .withValueClass(String.class)); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java index 43992120c84e..487c2a1930d4 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/Plugin.java @@ -73,6 +73,7 @@ public abstract class Plugin { /** Gets InputFormatProvider or OutputFormatProvider class for a plugin. */ public @Nullable abstract Class getFormatProviderClass(); + /** Gets Spark {@link Receiver} class for a CDAP plugin. */ public @Nullable abstract Class> getReceiverClass(); /** @@ -81,6 +82,10 @@ public abstract class Plugin { */ public @Nullable abstract SerializableFunction getGetOffsetFn(); + /** + * Gets a {@link SerializableFunction} that defines how to get constructor arguments for {@link + * Receiver} using {@link PluginConfig}. + */ public @Nullable abstract SerializableFunction getGetReceiverArgsFromConfigFn(); @@ -202,6 +207,7 @@ public static PluginConstants.PluginType initPluginType(Class pluginClass) } } + /** Initializes {@link BatchContextImpl} for CDAP plugin. */ public static BatchContextImpl initContext(Class cdapPluginClass) { // Init context and determine input or output Class contextClass; @@ -261,7 +267,13 @@ public Boolean isUnbounded() { .withConstructorArgs(getReceiverArgsFromConfigFn.apply(pluginConfig)); } - /** Creates a batch plugin instance. */ + /** + * Creates a batch plugin instance. + * + * @param newPluginClass class of the CDAP plugin {@link io.cdap.cdap.api.annotation.Plugin}. + * @param newFormatClass Hadoop Input or Output format class. + * @param newFormatProviderClass Hadoop Input or Output format provider class. + */ public static Plugin createBatch( Class newPluginClass, Class newFormatClass, Class newFormatProviderClass) { return Plugin.builder() @@ -273,7 +285,16 @@ public static Plugin createBatch( .build(); } - /** Creates a streaming plugin instance. */ + /** + * Creates a streaming plugin instance. + * + * @param newPluginClass class of the CDAP plugin {@link io.cdap.cdap.api.plugin.Plugin}. + * @param getOffsetFn {@link SerializableFunction} that defines how to get record offset for CDAP + * {@link io.cdap.cdap.api.annotation.Plugin} class. + * @param receiverClass Spark {@link Receiver} class for a CDAP plugin. + * @param getReceiverArgsFromConfigFn {@link SerializableFunction} that defines how to get + * constructor arguments for {@link Receiver} using {@link PluginConfig}. + */ public static Plugin createStreaming( Class newPluginClass, SerializableFunction getOffsetFn, @@ -292,6 +313,11 @@ public static Plugin createStreaming( /** * Creates a streaming plugin instance with default function for getting args for {@link * Receiver}. + * + * @param newPluginClass class of the CDAP plugin {@link io.cdap.cdap.api.plugin.Plugin}. + * @param getOffsetFn {@link SerializableFunction} that defines how to get record offset for CDAP + * {@link io.cdap.cdap.api.annotation.Plugin} class. + * @param receiverClass Spark {@link Receiver} class for a CDAP plugin. */ public static Plugin createStreaming( Class newPluginClass, From 44e70ab4aaa31f9aed4df5c60bf004095b9b6366 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Thu, 10 Nov 2022 14:02:14 +0400 Subject: [PATCH 11/21] Add pull frequency option --- .../cdap/CdapHubspotStreamingToTxt.java | 16 +++++---- .../complete/cdap/CdapHubspotToTxt.java | 3 +- .../complete/cdap/TxtToCdapHubspot.java | 3 +- .../cdap/options/BaseCdapOptions.java | 36 +++++++++++++++++++ .../cdap/options/CdapHubspotOptions.java | 12 ++----- .../options/CdapHubspotSourceOptions.java | 4 +-- .../CdapHubspotStreamingSourceOptions.java | 33 +++++++++++++++++ .../cdap/options/CdapSalesforceOptions.java | 12 ++----- .../cdap/options/CdapServiceNowOptions.java | 10 +----- .../cdap/options/CdapZendeskOptions.java | 10 +----- .../cdap/transforms/FormatInputTransform.java | 25 +++++++------ .../utils/PluginConfigOptionsConverter.java | 11 +++--- .../org/apache/beam/sdk/io/cdap/CdapIO.java | 23 +++++++++++- .../apache/beam/sdk/io/cdap/CdapIOTest.java | 10 +++++- .../ReadFromSparkReceiverWithOffsetDoFn.java | 22 +++++++++++- .../sdk/io/sparkreceiver/SparkReceiverIO.java | 14 ++++++++ .../io/sparkreceiver/SparkReceiverIOTest.java | 14 ++++++++ 17 files changed, 188 insertions(+), 70 deletions(-) create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/BaseCdapOptions.java create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index 58b0f12c2742..ad15ec83f5c1 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -19,7 +19,7 @@ import com.google.gson.JsonElement; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotSourceOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotStreamingSourceOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; @@ -94,8 +94,10 @@ public class CdapHubspotStreamingToTxt { * @param args Command line arguments to the pipeline. */ public static void main(String[] args) { - CdapHubspotSourceOptions options = - PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapHubspotSourceOptions.class); + CdapHubspotStreamingSourceOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(CdapHubspotStreamingSourceOptions.class); // Create the pipeline Pipeline pipeline = Pipeline.create(options); @@ -107,9 +109,8 @@ public static void main(String[] args) { * * @param options arguments to the pipeline */ - public static PipelineResult run(Pipeline pipeline, CdapHubspotSourceOptions options) { - Map paramsMap = - PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, true); + public static PipelineResult run(Pipeline pipeline, CdapHubspotStreamingSourceOptions options) { + Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options); LOG.info("Starting Cdap-Hubspot-streaming-to-txt pipeline with parameters: {}", paramsMap); /* @@ -123,7 +124,8 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotSourceOptions opt pipeline .apply( "readFromCdapHubspotStreaming", - FormatInputTransform.readFromCdapHubspotStreaming(paramsMap)) + FormatInputTransform.readFromCdapHubspotStreaming( + paramsMap, options.getPullFrequencySec())) .setCoder( KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java index 3f29adf50005..edc1779b0187 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java @@ -105,8 +105,7 @@ public static void main(String[] args) { * @param options arguments to the pipeline */ public static PipelineResult run(Pipeline pipeline, CdapHubspotSourceOptions options) { - Map paramsMap = - PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); + Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options); LOG.info("Starting Cdap-Hubspot-to-txt pipeline with parameters: {}", paramsMap); /* diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java index 2deaa544a465..6d7edd9c151d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java @@ -104,8 +104,7 @@ public static void main(String[] args) { * @param options arguments to the pipeline */ public static PipelineResult run(Pipeline pipeline, CdapHubspotSinkOptions options) { - Map paramsMap = - PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options, false); + Map paramsMap = PluginConfigOptionsConverter.hubspotOptionsToParamsMap(options); LOG.info("Starting Txt-to-Cdap-Hubspot pipeline with parameters: {}", paramsMap); /* diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/BaseCdapOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/BaseCdapOptions.java new file mode 100644 index 000000000000..a45cdb4320f0 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/BaseCdapOptions.java @@ -0,0 +1,36 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import io.cdap.plugin.common.Constants; +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.PipelineOptions; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link BaseCdapOptions} interface provides the custom execution options passed by the + * executor at the command-line for examples with Cdap plugins. + */ +public interface BaseCdapOptions extends PipelineOptions { + + @Validation.Required + @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) + String getReferenceName(); + + void setReferenceName(String referenceName); +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java index aae31a2657bf..bb49e0311ed7 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java @@ -17,16 +17,14 @@ */ package org.apache.beam.examples.complete.cdap.options; -import io.cdap.plugin.common.Constants; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; /** * The {@link CdapHubspotOptions} interface provides the custom execution options passed by the - * executor at the command-line. + * executor at the command-line for examples with Cdap Hubspot plugins. */ -public interface CdapHubspotOptions extends PipelineOptions { +public interface CdapHubspotOptions extends BaseCdapOptions { @Description("Hubspot api server url. If not specified then the default url will be used.") String getApiServerUrl(); @@ -44,10 +42,4 @@ public interface CdapHubspotOptions extends PipelineOptions { String getObjectType(); void setObjectType(String objectType); - - @Validation.Required - @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) - String getReferenceName(); - - void setReferenceName(String referenceName); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java index 755e80d3aec9..a74cf19380d1 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotSourceOptions.java @@ -22,9 +22,7 @@ /** * The {@link CdapHubspotSourceOptions} interface provides the custom execution options passed by - * the executor at the command-line for {@link - * org.apache.beam.examples.complete.cdap.CdapHubspotToTxt} and {@link - * org.apache.beam.examples.complete.cdap.CdapHubspotStreamingToTxt} examples. + * the executor at the command-line for Cdap Hubspot Source examples. */ public interface CdapHubspotSourceOptions extends CdapHubspotOptions { diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java new file mode 100644 index 000000000000..42cba33575a1 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java @@ -0,0 +1,33 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap.options; + +import org.apache.beam.sdk.options.Description; + +/** + * The {@link CdapHubspotSourceOptions} interface provides the custom execution options passed by + * the executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.CdapHubspotStreamingToTxt} example. + */ +public interface CdapHubspotStreamingSourceOptions extends CdapHubspotSourceOptions { + + @Description("Delay in seconds between polling for new records updates.") + Long getPullFrequencySec(); + + void setPullFrequencySec(Long pullFrequencySec); +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java index efe612671077..9997d0ea1d9e 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceOptions.java @@ -17,23 +17,15 @@ */ package org.apache.beam.examples.complete.cdap.options; -import io.cdap.plugin.common.Constants; import io.cdap.plugin.salesforce.SalesforceConstants; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; /** * The {@link CdapSalesforceOptions} interface provides the custom execution options passed by the - * executor at the command-line. + * executor at the command-line for example with Cdap Salesfroce plugins. */ -public interface CdapSalesforceOptions extends PipelineOptions { - - @Validation.Required - @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) - String getReferenceName(); - - void setReferenceName(String referenceName); +public interface CdapSalesforceOptions extends BaseCdapOptions { @Validation.Required @Description(SalesforceConstants.PROPERTY_USERNAME) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java index b06494d8f14d..bd8a84a352cd 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapServiceNowOptions.java @@ -17,9 +17,7 @@ */ package org.apache.beam.examples.complete.cdap.options; -import io.cdap.plugin.common.Constants; import org.apache.beam.sdk.options.Description; -import org.apache.beam.sdk.options.PipelineOptions; import org.apache.beam.sdk.options.Validation; /** @@ -27,7 +25,7 @@ * executor at the command-line for {@link * org.apache.beam.examples.complete.cdap.CdapServiceNowToTxt} example. */ -public interface CdapServiceNowOptions extends PipelineOptions { +public interface CdapServiceNowOptions extends BaseCdapOptions { @Validation.Required @Description("The Client ID for ServiceNow Instance.") @@ -87,12 +85,6 @@ public interface CdapServiceNowOptions extends PipelineOptions { void setValueType(String valueType); - @Validation.Required - @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) - String getReferenceName(); - - void setReferenceName(String referenceName); - @Validation.Required @Description( "Path to output folder with filename prefix." diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java index 4e9a6bd2e7e7..609b4be378e0 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapZendeskOptions.java @@ -17,10 +17,8 @@ */ package org.apache.beam.examples.complete.cdap.options; -import io.cdap.plugin.common.Constants; 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; /** @@ -28,7 +26,7 @@ * executor at the command-line for {@link org.apache.beam.examples.complete.cdap.CdapZendeskToTxt} * example. */ -public interface CdapZendeskOptions extends PipelineOptions { +public interface CdapZendeskOptions extends BaseCdapOptions { @Validation.Required @Description("Zendesk base url.") @@ -97,10 +95,4 @@ public interface CdapZendeskOptions extends PipelineOptions { String getOutputTxtFilePathPrefix(); void setOutputTxtFilePathPrefix(String outputTxtFilePathPrefix); - - @Validation.Required - @Description(Constants.Reference.REFERENCE_NAME_DESCRIPTION) - String getReferenceName(); - - void setReferenceName(String referenceName); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index 8cd9e646d4fb..c1be6704f807 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -93,7 +93,7 @@ public static CdapIO.Read readFromCdapHubspot( * @return configured Read transform */ public static CdapIO.Read readFromCdapHubspotStreaming( - Map pluginConfigParams) { + Map pluginConfigParams, Long pullFrequencySec) { final HubspotStreamingSourceConfig pluginConfig = new ConfigWrapper<>(HubspotStreamingSourceConfig.class) @@ -101,15 +101,20 @@ public static CdapIO.Read readFromCdapHubspotStreaming( .build(); checkStateNotNull(pluginConfig, "Plugin config can't be null."); - return CdapIO.read() - .withCdapPlugin( - Plugin.createStreaming( - HubspotStreamingSource.class, - GetOffsetUtils.getOffsetFnForHubspot(), - HubspotReceiver.class)) - .withPluginConfig(pluginConfig) - .withKeyClass(NullWritable.class) - .withValueClass(String.class); + CdapIO.Read read = + CdapIO.read() + .withCdapPlugin( + Plugin.createStreaming( + HubspotStreamingSource.class, + GetOffsetUtils.getOffsetFnForHubspot(), + HubspotReceiver.class)) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(String.class); + if (pullFrequencySec != null) { + read = read.withPullFrequencySec(pullFrequencySec); + } + return read; } /** diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index dccf9c2414fd..1ee536c1ea59 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -30,6 +30,7 @@ import io.cdap.plugin.zendesk.source.common.config.BaseZendeskSourceConfig; import java.util.Map; import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotStreamingSourceOptions; import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSinkOptions; import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSourceOptions; import org.apache.beam.examples.complete.cdap.options.CdapServiceNowOptions; @@ -42,8 +43,7 @@ */ public class PluginConfigOptionsConverter { - public static Map hubspotOptionsToParamsMap( - CdapHubspotOptions options, boolean isStreaming) { + public static Map hubspotOptionsToParamsMap(CdapHubspotOptions options) { String apiServerUrl = options.getApiServerUrl(); ImmutableMap.Builder builder = ImmutableMap.builder() @@ -53,9 +53,10 @@ public static Map hubspotOptionsToParamsMap( .put(BaseHubspotConfig.API_KEY, options.getApiKey()) .put(BaseHubspotConfig.OBJECT_TYPE, options.getObjectType()) .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()); - if (isStreaming) { - // Pull frequency is not implemented in CdapIO, but still needs to be passed for - // HubspotStreamingSource plugin + if (options instanceof CdapHubspotStreamingSourceOptions) { + // Hubspot PullFrequency value will be ignored as pull frequency is implemented differently in + // CdapIO, + // but it still needs to be passed for the plugin to work correctly. builder.put(HubspotStreamingSourceConfig.PULL_FREQUENCY, PullFrequency.MINUTES_15.getName()); } return builder.build(); diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java index 3da7c4deb749..8255ce2938e5 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java @@ -148,6 +148,9 @@ *

To configure {@link CdapIO} source, you must specify Cdap {@link Plugin}, Cdap {@link * PluginConfig}, key and value classes. * + *

Optionally you can pass {@code pullFrequencySec} which is a delay in seconds between polling + * for new records updates. + * *

{@link Plugin} is the Wrapper class for the Cdap Plugin. It contains main information about * the Plugin. The object of the {@link Plugin} class can be created with the {@link * Plugin#createStreaming(Class, SerializableFunction, Class)} method. Method requires {@link @@ -178,7 +181,8 @@ * EmployeeReceiver.class)) * .withPluginConfig(pluginConfig) * .withKeyClass(String.class) - * .withValueClass(String.class)); + * .withValueClass(String.class) + * .withPullFrequencySec(1L); * } */ @Experimental(Kind.SOURCE_SINK) @@ -217,6 +221,8 @@ public abstract static class Read extends PTransform getValueClass(); + abstract @Nullable Long getPullFrequencySec(); + abstract Builder toBuilder(); @Experimental(Experimental.Kind.PORTABILITY) @@ -231,6 +237,8 @@ abstract static class Builder { abstract Builder setValueClass(Class valueClass); + abstract Builder setPullFrequencySec(Long pullFrequencySec); + abstract Read build(); } @@ -265,6 +273,15 @@ public Read withValueClass(Class valueClass) { return toBuilder().setValueClass(valueClass).build(); } + /** + * Delay in seconds between polling for new records updates. Applicable only for streaming Cdap + * Plugins. + */ + public Read withPullFrequencySec(Long pullFrequencySec) { + checkArgument(pullFrequencySec != null, "Pull frequency can not be null"); + return toBuilder().setPullFrequencySec(pullFrequencySec).build(); + } + @Override public PCollection> expand(PBegin input) { Plugin cdapPlugin = getCdapPlugin(); @@ -291,6 +308,10 @@ public PCollection> expand(PBegin input) { SparkReceiverIO.read() .withGetOffsetFn(getOffsetFn) .withSparkReceiverBuilder(receiverBuilder); + Long pullFrequencySec = getPullFrequencySec(); + if (pullFrequencySec != null) { + reader = reader.withPullFrequencySec(pullFrequencySec); + } try { Coder coder = input.getPipeline().getCoderRegistry().getCoder(valueClass); PCollection values = input.apply(reader).setCoder(coder); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java index 6fcfdbdfc24d..399d92688fd9 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java @@ -136,6 +136,13 @@ public void testReadObjectCreationFailsIfValueClassIsNull() { IllegalArgumentException.class, () -> CdapIO.read().withValueClass(null)); } + @Test + public void testReadObjectCreationFailsIfPullFrequencySecIsNull() { + assertThrows( + IllegalArgumentException.class, + () -> CdapIO.read().withPullFrequencySec(null)); + } + @Test public void testReadExpandingFailsMissingCdapPluginClass() { PBegin testPBegin = PBegin.in(TestPipeline.create()); @@ -194,7 +201,8 @@ public void testReadFromCdapStreamingPlugin() { config -> new Object[] {config})) .withPluginConfig(pluginConfig) .withKeyClass(String.class) - .withValueClass(String.class); + .withValueClass(String.class) + .withPullFrequencySec(1L); List storedRecords = EmployeeReceiver.getStoredRecords(); diff --git a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java index 8b2fdcb01ad1..325e14c52cf9 100644 --- a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java +++ b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java @@ -63,13 +63,17 @@ class ReadFromSparkReceiverWithOffsetDoFn extends DoFn { LoggerFactory.getLogger(ReadFromSparkReceiverWithOffsetDoFn.class); /** Constant waiting time after the {@link Receiver} starts. Required to prepare for polling */ - private static final int START_POLL_TIMEOUT_MS = 2000; + private static final long START_POLL_TIMEOUT_MS = 2000; + + /** Delay between polling for new records updates. */ + private static final long DEFAULT_PULL_FREQUENCY_SEC = 0; private final SerializableFunction> createWatermarkEstimatorFn; private final SerializableFunction getOffsetFn; private final SerializableFunction getTimestampFn; private final ReceiverBuilder> sparkReceiverBuilder; + private final Long pullFrequencySec; ReadFromSparkReceiverWithOffsetDoFn(SparkReceiverIO.Read transform) { createWatermarkEstimatorFn = WatermarkEstimators.Manual::new; @@ -88,6 +92,12 @@ class ReadFromSparkReceiverWithOffsetDoFn extends DoFn { getTimestampFn = input -> Instant.now(); } this.getTimestampFn = getTimestampFn; + + Long pullFrequencySec = transform.getPullFrequencySec(); + if (pullFrequencySec == null) { + pullFrequencySec = DEFAULT_PULL_FREQUENCY_SEC; + } + this.pullFrequencySec = pullFrequencySec; } @GetInitialRestriction @@ -298,6 +308,16 @@ public ProcessContinuation processElement( if (!sparkConsumer.hasRecords()) { sparkConsumer.stop(); tracker.checkDone(); + if (pullFrequencySec != 0L) { + LOG.debug("Waiting to poll for new records..."); + try { + TimeUnit.SECONDS.sleep(pullFrequencySec); + } catch (InterruptedException e) { + LOG.error("SparkReceiver was interrupted while waiting to poll new records", e); + throw new IllegalStateException( + "Spark Receiver was interrupted while waiting to poll new records"); + } + } LOG.debug("Resume for restriction: {}", tracker.currentRestriction().toString()); return ProcessContinuation.resume(); } diff --git a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java index 954ce2b836b3..c5bbee2217bb 100644 --- a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java +++ b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java @@ -50,6 +50,9 @@ *

Optionally you can pass {@code timestampFn} which is a {@link SerializableFunction} that * defines how to get {@code Instant timestamp} from {@code V record}. * + *

Optionally you can pass {@code pullFrequencySec} which is a delay in seconds between polling + * for new records updates. + * *

Example of {@link SparkReceiverIO#read()} usage: * *

{@code
@@ -65,6 +68,7 @@
  *    SparkReceiverIO.read()
  *      .withGetOffsetFn(Long::valueOf)
  *      .withTimestampFn(Instant::parse)
+ *      .withPullFrequencySec(1L)
  *      .withSparkReceiverBuilder(receiverBuilder);
  * }
*/ @@ -88,6 +92,8 @@ public abstract static class Read extends PTransform> abstract @Nullable SerializableFunction getTimestampFn(); + abstract @Nullable Long getPullFrequencySec(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -100,6 +106,8 @@ abstract Builder setSparkReceiverBuilder( abstract Builder setTimestampFn(SerializableFunction timestampFn); + abstract Builder setPullFrequencySec(Long pullFrequencySec); + abstract Read build(); } @@ -122,6 +130,12 @@ public Read withTimestampFn(SerializableFunction timestampFn) { return toBuilder().setTimestampFn(timestampFn).build(); } + /** Delay in seconds between polling for new records updates. */ + public Read withPullFrequencySec(Long pullFrequencySec) { + checkArgument(pullFrequencySec != null, "Pull frequency can not be null"); + return toBuilder().setPullFrequencySec(pullFrequencySec).build(); + } + @Override public PCollection expand(PBegin input) { validateTransform(); diff --git a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java index 6931e7199926..b429956c7fab 100644 --- a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java +++ b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java @@ -41,6 +41,7 @@ public class SparkReceiverIOTest { public static final TestPipelineOptions OPTIONS = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); + public static final long PULL_FREQUENCY_SEC = 1L; static { OPTIONS.setBlockOnRun(false); @@ -59,6 +60,7 @@ public void testReadBuildsCorrectly() { SparkReceiverIO.read() .withGetOffsetFn(offsetFn) .withTimestampFn(timestampFn) + .withPullFrequencySec(1L) .withSparkReceiverBuilder(receiverBuilder); assertEquals(offsetFn, read.getGetOffsetFn()); @@ -84,6 +86,13 @@ public void testReadObjectCreationFailsIfTimestampFnIsNull() { IllegalArgumentException.class, () -> SparkReceiverIO.read().withTimestampFn(null)); } + @Test + public void testReadObjectCreationFailsIfPullFrequencySecIsNull() { + assertThrows( + IllegalArgumentException.class, + () -> SparkReceiverIO.read().withPullFrequencySec(null)); + } + @Test public void testReadValidationFailsMissingReceiverBuilder() { SparkReceiverIO.Read read = SparkReceiverIO.read(); @@ -108,6 +117,7 @@ public void testReadFromCustomReceiverWithOffset() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) + .withPullFrequencySec(1L) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -129,6 +139,7 @@ public void testReadFromCustomReceiverWithOffsetFailsAndReread() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) + .withPullFrequencySec(1L) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -149,6 +160,7 @@ public void testReadFromReceiverArrayBufferData() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) + .withPullFrequencySec(1L) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -169,6 +181,7 @@ public void testReadFromReceiverByteBufferData() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) + .withPullFrequencySec(1L) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -189,6 +202,7 @@ public void testReadFromReceiverIteratorData() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) + .withPullFrequencySec(PULL_FREQUENCY_SEC) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); From ab344e1f10652725919a80182627b4a52177ca08 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Mon, 14 Nov 2022 11:16:28 +0400 Subject: [PATCH 12/21] Add startOffset option --- .../cdap/CdapHubspotStreamingToTxt.java | 2 +- .../beam/examples/complete/cdap/README.md | 6 ++++- .../CdapHubspotStreamingSourceOptions.java | 5 ++++ .../cdap/transforms/FormatInputTransform.java | 7 +++++- .../org/apache/beam/sdk/io/cdap/CdapIO.java | 23 +++++++++++++++++-- .../apache/beam/sdk/io/cdap/CdapIOTest.java | 9 +++++++- .../ReadFromSparkReceiverWithOffsetDoFn.java | 15 ++++++++++-- .../sdk/io/sparkreceiver/SparkReceiverIO.java | 14 ++++++++++- .../io/sparkreceiver/SparkReceiverIOTest.java | 23 +++++++++++++++---- 9 files changed, 90 insertions(+), 14 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index ad15ec83f5c1..4fc5180b497d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -125,7 +125,7 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotStreamingSourceOp .apply( "readFromCdapHubspotStreaming", FormatInputTransform.readFromCdapHubspotStreaming( - paramsMap, options.getPullFrequencySec())) + paramsMap, options.getPullFrequencySec(), options.getStartOffset())) .setCoder( KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 7fb96d46282d..6703e54c65ac 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -77,6 +77,8 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap - `apikey` - Hubspot OAuth2 API Key - `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) - `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. +- `pullFrequencySec` - delay in seconds between polling for new records updates. (Optional) +- `startOffset` - inclusive start offset from which the reading should be started. (Optional) Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. @@ -86,7 +88,9 @@ To execute this pipeline, specify the parameters in the following format: --apikey=your-api-key \ --referenceName=your-reference-name \ --objectType=Contacts \ - --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix + --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \ + --pullFrequencySec=100 \ + --startOffset=1000 ``` ## Running the TxtToCdapHubspot pipeline example diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java index 42cba33575a1..4feb4d1e0f42 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotStreamingSourceOptions.java @@ -30,4 +30,9 @@ public interface CdapHubspotStreamingSourceOptions extends CdapHubspotSourceOpti Long getPullFrequencySec(); void setPullFrequencySec(Long pullFrequencySec); + + @Description("Inclusive start offset from which the reading should be started.") + Long getStartOffset(); + + void setStartOffset(Long startOffset); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index c1be6704f807..e5525cf88590 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -90,10 +90,12 @@ public static CdapIO.Read readFromCdapHubspot( * Configures Cdap Hubspot Streaming Read transform. * * @param pluginConfigParams Cdap Hubspot plugin config parameters + * @param pullFrequencySec Delay in seconds between polling for new records updates + * @param startOffset Inclusive start offset from which the reading should be started * @return configured Read transform */ public static CdapIO.Read readFromCdapHubspotStreaming( - Map pluginConfigParams, Long pullFrequencySec) { + Map pluginConfigParams, Long pullFrequencySec, Long startOffset) { final HubspotStreamingSourceConfig pluginConfig = new ConfigWrapper<>(HubspotStreamingSourceConfig.class) @@ -114,6 +116,9 @@ public static CdapIO.Read readFromCdapHubspotStreaming( if (pullFrequencySec != null) { read = read.withPullFrequencySec(pullFrequencySec); } + if (startOffset != null) { + read = read.withStartOffset(startOffset); + } return read; } diff --git a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java index 8255ce2938e5..13aac53f7246 100644 --- a/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java +++ b/sdks/java/io/cdap/src/main/java/org/apache/beam/sdk/io/cdap/CdapIO.java @@ -149,7 +149,8 @@ * PluginConfig}, key and value classes. * *

Optionally you can pass {@code pullFrequencySec} which is a delay in seconds between polling - * for new records updates. + * for new records updates, you can pass {@code startOffset} which is inclusive start offset from + * which the reading should be started. * *

{@link Plugin} is the Wrapper class for the Cdap Plugin. It contains main information about * the Plugin. The object of the {@link Plugin} class can be created with the {@link @@ -182,7 +183,8 @@ * .withPluginConfig(pluginConfig) * .withKeyClass(String.class) * .withValueClass(String.class) - * .withPullFrequencySec(1L); + * .withPullFrequencySec(1L) + * .withStartOffset(10L); * } */ @Experimental(Kind.SOURCE_SINK) @@ -223,6 +225,8 @@ public abstract static class Read extends PTransform toBuilder(); @Experimental(Experimental.Kind.PORTABILITY) @@ -239,6 +243,8 @@ abstract static class Builder { abstract Builder setPullFrequencySec(Long pullFrequencySec); + abstract Builder setStartOffset(Long startOffset); + abstract Read build(); } @@ -282,6 +288,15 @@ public Read withPullFrequencySec(Long pullFrequencySec) { return toBuilder().setPullFrequencySec(pullFrequencySec).build(); } + /** + * Inclusive start offset from which the reading should be started. Applicable only for + * streaming Cdap Plugins. + */ + public Read withStartOffset(Long startOffset) { + checkArgument(startOffset != null, "Start offset can not be null"); + return toBuilder().setStartOffset(startOffset).build(); + } + @Override public PCollection> expand(PBegin input) { Plugin cdapPlugin = getCdapPlugin(); @@ -312,6 +327,10 @@ public PCollection> expand(PBegin input) { if (pullFrequencySec != null) { reader = reader.withPullFrequencySec(pullFrequencySec); } + Long startOffset = getStartOffset(); + if (startOffset != null) { + reader = reader.withStartOffset(startOffset); + } try { Coder coder = input.getPipeline().getCoderRegistry().getCoder(valueClass); PCollection values = input.apply(reader).setCoder(coder); diff --git a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java index 399d92688fd9..5fe5d7bf2743 100644 --- a/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java +++ b/sdks/java/io/cdap/src/test/java/org/apache/beam/sdk/io/cdap/CdapIOTest.java @@ -143,6 +143,12 @@ public void testReadObjectCreationFailsIfPullFrequencySecIsNull() { () -> CdapIO.read().withPullFrequencySec(null)); } + @Test + public void testReadObjectCreationFailsIfStartOffsetIsNull() { + assertThrows( + IllegalArgumentException.class, () -> CdapIO.read().withStartOffset(null)); + } + @Test public void testReadExpandingFailsMissingCdapPluginClass() { PBegin testPBegin = PBegin.in(TestPipeline.create()); @@ -202,7 +208,8 @@ public void testReadFromCdapStreamingPlugin() { .withPluginConfig(pluginConfig) .withKeyClass(String.class) .withValueClass(String.class) - .withPullFrequencySec(1L); + .withPullFrequencySec(1L) + .withStartOffset(0L); List storedRecords = EmployeeReceiver.getStoredRecords(); diff --git a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java index 325e14c52cf9..b6276670b4d8 100644 --- a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java +++ b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/ReadFromSparkReceiverWithOffsetDoFn.java @@ -51,7 +51,8 @@ * A SplittableDoFn which reads from {@link Receiver} that implements {@link HasOffset}. By default, * a {@link WatermarkEstimators.Manual} watermark estimator is used to track watermark. * - *

Initial range The initial range is {@code [0, Long.MAX_VALUE)} + *

By default the initial range is {@code [0, Long.MAX_VALUE)}. There is an ability to set {@code + * startOffset}. * *

Resume Processing Every time the sparkConsumer.hasRecords() returns false, {@link * ReadFromSparkReceiverWithOffsetDoFn} will move to process the next element. @@ -68,12 +69,16 @@ class ReadFromSparkReceiverWithOffsetDoFn extends DoFn { /** Delay between polling for new records updates. */ private static final long DEFAULT_PULL_FREQUENCY_SEC = 0; + /** Inclusive start offset from which the reading should be started. */ + private static final long DEFAULT_START_OFFSET = 0; + private final SerializableFunction> createWatermarkEstimatorFn; private final SerializableFunction getOffsetFn; private final SerializableFunction getTimestampFn; private final ReceiverBuilder> sparkReceiverBuilder; private final Long pullFrequencySec; + private final Long startOffset; ReadFromSparkReceiverWithOffsetDoFn(SparkReceiverIO.Read transform) { createWatermarkEstimatorFn = WatermarkEstimators.Manual::new; @@ -98,11 +103,17 @@ class ReadFromSparkReceiverWithOffsetDoFn extends DoFn { pullFrequencySec = DEFAULT_PULL_FREQUENCY_SEC; } this.pullFrequencySec = pullFrequencySec; + + Long startOffset = transform.getStartOffset(); + if (startOffset == null) { + startOffset = DEFAULT_START_OFFSET; + } + this.startOffset = pullFrequencySec; } @GetInitialRestriction public OffsetRange initialRestriction(@Element byte[] element) { - return new OffsetRange(0, Long.MAX_VALUE); + return new OffsetRange(startOffset, Long.MAX_VALUE); } @GetInitialWatermarkEstimatorState diff --git a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java index c5bbee2217bb..4982981cdae8 100644 --- a/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java +++ b/sdks/java/io/sparkreceiver/2/src/main/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIO.java @@ -48,7 +48,8 @@ * {@code Long offset} from {@code V record}. * *

Optionally you can pass {@code timestampFn} which is a {@link SerializableFunction} that - * defines how to get {@code Instant timestamp} from {@code V record}. + * defines how to get {@code Instant timestamp} from {@code V record}, you can pass {@code + * startOffset} which is inclusive start offset from which the reading should be started. * *

Optionally you can pass {@code pullFrequencySec} which is a delay in seconds between polling * for new records updates. @@ -69,6 +70,7 @@ * .withGetOffsetFn(Long::valueOf) * .withTimestampFn(Instant::parse) * .withPullFrequencySec(1L) + * .withStartOffset(10L) * .withSparkReceiverBuilder(receiverBuilder); * } */ @@ -94,6 +96,8 @@ public abstract static class Read extends PTransform> abstract @Nullable Long getPullFrequencySec(); + abstract @Nullable Long getStartOffset(); + abstract Builder toBuilder(); @AutoValue.Builder @@ -108,6 +112,8 @@ abstract Builder setSparkReceiverBuilder( abstract Builder setPullFrequencySec(Long pullFrequencySec); + abstract Builder setStartOffset(Long startOffset); + abstract Read build(); } @@ -136,6 +142,12 @@ public Read withPullFrequencySec(Long pullFrequencySec) { return toBuilder().setPullFrequencySec(pullFrequencySec).build(); } + /** Inclusive start offset from which the reading should be started. */ + public Read withStartOffset(Long startOffset) { + checkArgument(startOffset != null, "Start offset can not be null"); + return toBuilder().setStartOffset(startOffset).build(); + } + @Override public PCollection expand(PBegin input) { validateTransform(); diff --git a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java index b429956c7fab..a666aa6417dd 100644 --- a/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java +++ b/sdks/java/io/sparkreceiver/2/src/test/java/org/apache/beam/sdk/io/sparkreceiver/SparkReceiverIOTest.java @@ -42,6 +42,7 @@ public class SparkReceiverIOTest { public static final TestPipelineOptions OPTIONS = TestPipeline.testingPipelineOptions().as(TestPipelineOptions.class); public static final long PULL_FREQUENCY_SEC = 1L; + public static final long START_OFFSET = 0L; static { OPTIONS.setBlockOnRun(false); @@ -60,7 +61,8 @@ public void testReadBuildsCorrectly() { SparkReceiverIO.read() .withGetOffsetFn(offsetFn) .withTimestampFn(timestampFn) - .withPullFrequencySec(1L) + .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); assertEquals(offsetFn, read.getGetOffsetFn()); @@ -93,6 +95,12 @@ public void testReadObjectCreationFailsIfPullFrequencySecIsNull() { () -> SparkReceiverIO.read().withPullFrequencySec(null)); } + @Test + public void testReadObjectCreationFailsIfStartOffsetIsNull() { + assertThrows( + IllegalArgumentException.class, () -> SparkReceiverIO.read().withStartOffset(null)); + } + @Test public void testReadValidationFailsMissingReceiverBuilder() { SparkReceiverIO.Read read = SparkReceiverIO.read(); @@ -117,7 +125,8 @@ public void testReadFromCustomReceiverWithOffset() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) - .withPullFrequencySec(1L) + .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -139,7 +148,8 @@ public void testReadFromCustomReceiverWithOffsetFailsAndReread() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) - .withPullFrequencySec(1L) + .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -160,7 +170,8 @@ public void testReadFromReceiverArrayBufferData() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) - .withPullFrequencySec(1L) + .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -181,7 +192,8 @@ public void testReadFromReceiverByteBufferData() { SparkReceiverIO.read() .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) - .withPullFrequencySec(1L) + .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); @@ -203,6 +215,7 @@ public void testReadFromReceiverIteratorData() { .withGetOffsetFn(Long::valueOf) .withTimestampFn(Instant::parse) .withPullFrequencySec(PULL_FREQUENCY_SEC) + .withStartOffset(START_OFFSET) .withSparkReceiverBuilder(receiverBuilder); List expected = new ArrayList<>(); From 678d4c29f55356b5af63e7a8c9854950c07f58bd Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Mon, 14 Nov 2022 18:58:38 +0400 Subject: [PATCH 13/21] Add Salesforce Streaming example --- examples/java/cdap/build.gradle | 4 +- .../cdap/CdapHubspotStreamingToTxt.java | 7 +- .../cdap/CdapSalesforceStreamingToTxt.java | 160 ++++++++++++++++++ .../complete/cdap/CdapSalesforceToTxt.java | 2 +- .../complete/cdap/TxtToCdapSalesforce.java | 4 +- .../CdapSalesforceStreamingSourceOptions.java | 45 +++++ .../cdap/transforms/FormatInputTransform.java | 46 +++++ .../complete/cdap/utils/GetOffsetUtils.java | 19 +++ .../utils/PluginConfigOptionsConverter.java | 32 +++- 9 files changed, 303 insertions(+), 16 deletions(-) create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java create mode 100644 examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceStreamingSourceOptions.java diff --git a/examples/java/cdap/build.gradle b/examples/java/cdap/build.gradle index 610ea573cb91..b070c95a5548 100644 --- a/examples/java/cdap/build.gradle +++ b/examples/java/cdap/build.gradle @@ -59,7 +59,9 @@ dependencies { implementation "com.akvelon:cdap-hubspot-plugins:1.0.3" implementation library.java.cdap_plugin_service_now implementation library.java.cdap_plugin_zendesk - implementation library.java.cdap_plugin_salesforce + //TODO: modify to 'implementation library.java.cdap_plugin_salesforce', + // when new release with HasOffset interface will be published + implementation "com.akvelon:cdap-salesforce-plugins:1.5.2" implementation library.java.google_code_gson implementation library.java.hadoop_common implementation library.java.slf4j_api diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index 4fc5180b497d..e39cfaaea072 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -17,11 +17,9 @@ */ package org.apache.beam.examples.complete.cdap; -import com.google.gson.JsonElement; import java.util.Map; import org.apache.beam.examples.complete.cdap.options.CdapHubspotStreamingSourceOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; -import org.apache.beam.examples.complete.cdap.utils.JsonElementCoder; import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; import org.apache.beam.sdk.Pipeline; import org.apache.beam.sdk.PipelineResult; @@ -74,7 +72,9 @@ * --apikey=your-api-key \ * --referenceName=your-reference-name \ * --objectType=Contacts \ - * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \ + * --pullFrequencySec=1 \ + * --startOffset=0 * } * * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: @@ -119,7 +119,6 @@ public static PipelineResult run(Pipeline pipeline, CdapHubspotStreamingSourceOp * 2) Extract values only * 3) Write successful records to .txt file */ - pipeline.getCoderRegistry().registerCoderForClass(JsonElement.class, JsonElementCoder.of()); pipeline .apply( diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java new file mode 100644 index 000000000000..2815e6ab92bd --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.beam.examples.complete.cdap; + +import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; +import java.util.Map; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceStreamingSourceOptions; +import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; +import org.apache.beam.examples.complete.cdap.utils.PluginConfigOptionsConverter; +import org.apache.beam.sdk.Pipeline; +import org.apache.beam.sdk.PipelineResult; +import org.apache.beam.sdk.coders.KvCoder; +import org.apache.beam.sdk.coders.NullableCoder; +import org.apache.beam.sdk.coders.StringUtf8Coder; +import org.apache.beam.sdk.io.TextIO; +import org.apache.beam.sdk.io.hadoop.WritableCoder; +import org.apache.beam.sdk.options.PipelineOptionsFactory; +import org.apache.beam.sdk.transforms.Values; +import org.apache.beam.sdk.transforms.windowing.AfterProcessingTime; +import org.apache.beam.sdk.transforms.windowing.GlobalWindows; +import org.apache.beam.sdk.transforms.windowing.Repeatedly; +import org.apache.beam.sdk.transforms.windowing.Window; +import org.apache.beam.sdk.values.KV; +import org.apache.hadoop.io.NullWritable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The {@link CdapSalesforceStreamingToTxt} pipeline is a streaming pipeline which ingests data in + * JSON format from CDAP Salesforce, and outputs the resulting records to .txt file. Salesforce + * parameters and output .txt file path are specified by the user as template parameters.
+ * + *

Example Usage + * + *

+ * # Gradle preparation
+ *
+ * To run this example your {@code build.gradle} file should contain the following task
+ * to execute the pipeline:
+ * {@code
+ * task executeCdap (type:JavaExec) {
+ *     mainClass = 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 executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapSalesforceStreamingToTxt \
+ *      -Dexec.args="--= --="
+ * }
+ *
+ * # Running the pipeline
+ * To execute this pipeline, specify the parameters in the following format:
+ * {@code
+ * --username=your-user-name\
+ * --password=your-password \
+ * --securityToken=your-token \
+ * --consumerKey=your-key \
+ * --consumerSecret=your-secret \
+ * --loginUrl=your-login-url \
+ * --sObjectName=object-name \
+ * --referenceName=your-reference-name \
+ * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \
+ * --pullFrequencySec=1 \
+ * --startOffset=0
+ * }
+ *
+ * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify:
+ * {@code
+ * --runner=YOUR_SELECTED_RUNNER
+ * }
+ * 
+ */ +public class CdapSalesforceStreamingToTxt { + + /* Logger for class.*/ + private static final Logger LOG = LoggerFactory.getLogger(CdapSalesforceStreamingToTxt.class); + + /** + * Main entry point for pipeline execution. + * + * @param args Command line arguments to the pipeline. + */ + public static void main(String[] args) { + CdapSalesforceStreamingSourceOptions options = + PipelineOptionsFactory.fromArgs(args) + .withValidation() + .as(CdapSalesforceStreamingSourceOptions.class); + + // Create the pipeline + Pipeline pipeline = Pipeline.create(options); + run(pipeline, options); + } + + /** + * Runs a pipeline which reads records from CDAP Salesforce and writes them to .txt file. + * + * @param options arguments to the pipeline + */ + public static PipelineResult run( + Pipeline pipeline, CdapSalesforceStreamingSourceOptions options) { + Map paramsMap = + PluginConfigOptionsConverter.salesforceStreamingSourceOptionsToParamsMap(options); + LOG.info("Starting Cdap-Salesforce-streaming-to-txt pipeline with parameters: {}", paramsMap); + + AuthenticatorCredentials salesforceCredentials = + PluginConfigOptionsConverter.getSalesforceStreamingAuthenticatorCredentials(options); + + /* + * Steps: + * 1) Read messages in from Cdap Salesforce + * 2) Extract values only + * 3) Write successful records to .txt file + */ + + pipeline + .apply( + "readFromCdapSalesforceStreaming", + FormatInputTransform.readFromCdapSalesforceStreaming( + paramsMap, + salesforceCredentials, + options.getPullFrequencySec(), + options.getStartOffset())) + .setCoder( + KvCoder.of( + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) + .apply( + "globalwindow", + Window.>into(new GlobalWindows()) + .triggering(Repeatedly.forever(AfterProcessingTime.pastFirstElementInPane())) + .discardingFiredPanes()) + .apply(Values.create()) + .apply( + "writeToTxt", + TextIO.write() + .withWindowedWrites() + .withNumShards(1) + .to(options.getOutputTxtFilePathPrefix())); + + return pipeline.run(); + } +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java index 43c6cbb82e62..196395cd958b 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceToTxt.java @@ -112,7 +112,7 @@ public static void main(String[] args) { @SuppressWarnings("rawtypes") public static PipelineResult run(Pipeline pipeline, CdapSalesforceSourceOptions options) { Map paramsMap = - PluginConfigOptionsConverter.salesforceSourceOptionsToParamsMap(options); + PluginConfigOptionsConverter.salesforceBatchSourceOptionsToParamsMap(options); LOG.info("Starting Cdap-Salesforce pipeline with parameters: {}", paramsMap); /* diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java index 337c463cdfc6..ae4f577cff89 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapSalesforce.java @@ -121,7 +121,7 @@ public static void main(String[] args) { */ public static PipelineResult run(Pipeline pipeline, CdapSalesforceSinkOptions options) { Map paramsMap = - PluginConfigOptionsConverter.salesforceSinkOptionsToParamsMap(options); + PluginConfigOptionsConverter.salesforceBatchSinkOptionsToParamsMap(options); LOG.info("Starting Txt-to-Cdap-Salesforce pipeline with parameters: {}", paramsMap); /* @@ -145,7 +145,7 @@ public static PipelineResult run(Pipeline pipeline, CdapSalesforceSinkOptions op .apply( "writeToCdapSalesforce", FormatOutputTransform.writeToCdapSalesforce( - PluginConfigOptionsConverter.salesforceSinkOptionsToParamsMap(options), + PluginConfigOptionsConverter.salesforceBatchSinkOptionsToParamsMap(options), options.getLocksDirPath())); return pipeline.run(); diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceStreamingSourceOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceStreamingSourceOptions.java new file mode 100644 index 000000000000..d41ea8f84280 --- /dev/null +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapSalesforceStreamingSourceOptions.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.examples.complete.cdap.options; + +import org.apache.beam.sdk.options.Description; +import org.apache.beam.sdk.options.Validation; + +/** + * The {@link CdapSalesforceStreamingSourceOptions} interface provides the custom execution options + * passed by the executor at the command-line for {@link + * org.apache.beam.examples.complete.cdap.CdapSalesforceStreamingToTxt} example. + */ +public interface CdapSalesforceStreamingSourceOptions extends CdapSalesforceSourceOptions { + + @Validation.Required + @Description("Salesforce push topic name. Plugin will track updates from this topic.") + String getPushTopicName(); + + void setPushTopicName(String pushTopicName); + + @Description("Delay in seconds between polling for new records updates.") + Long getPullFrequencySec(); + + void setPullFrequencySec(Long pullFrequencySec); + + @Description("Inclusive start offset from which the reading should be started.") + Long getStartOffset(); + + void setStartOffset(Long startOffset); +} diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index e5525cf88590..2ac529cec381 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -27,8 +27,12 @@ import io.cdap.plugin.hubspot.source.streaming.HubspotReceiver; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; +import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; +import io.cdap.plugin.salesforce.plugin.source.streaming.SalesforceReceiver; +import io.cdap.plugin.salesforce.plugin.source.streaming.SalesforceStreamingSource; +import io.cdap.plugin.salesforce.plugin.source.streaming.SalesforceStreamingSourceConfig; import io.cdap.plugin.servicenow.source.ServiceNowSource; import io.cdap.plugin.servicenow.source.ServiceNowSourceConfig; import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSource; @@ -164,4 +168,46 @@ public static CdapIO.Read readFromCdapSalesforce( .withKeyClass(Schema.class) .withValueClass(LinkedHashMap.class); } + + /** + * Configures Cdap Salesforce Streaming Read transform. + * + * @param pluginConfigParams Cdap Hubspot plugin config parameters + * @param salesforceCredentials Authenticator credentials for Salesforce + * @param pullFrequencySec Delay in seconds between polling for new records updates + * @param startOffset Inclusive start offset from which the reading should be started + * @return configured Read transform + */ + public static CdapIO.Read readFromCdapSalesforceStreaming( + Map pluginConfigParams, + AuthenticatorCredentials salesforceCredentials, + Long pullFrequencySec, + Long startOffset) { + + final SalesforceStreamingSourceConfig pluginConfig = + new ConfigWrapper<>(SalesforceStreamingSourceConfig.class) + .withParams(pluginConfigParams) + .build(); + checkStateNotNull(pluginConfig, "Plugin config can't be null."); + + CdapIO.Read read = + CdapIO.read() + .withCdapPlugin( + Plugin.createStreaming( + SalesforceStreamingSource.class, + GetOffsetUtils.getOffsetFnForSalesforce(), + SalesforceReceiver.class, + config -> + new Object[] {salesforceCredentials, pluginConfig.getPushTopicName()})) + .withPluginConfig(pluginConfig) + .withKeyClass(NullWritable.class) + .withValueClass(String.class); + if (pullFrequencySec != null) { + read = read.withPullFrequencySec(pullFrequencySec); + } + if (startOffset != null) { + read = read.withStartOffset(startOffset); + } + return read; + } } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java index a03f723694f9..12bdbcf1335d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java @@ -36,8 +36,27 @@ public class GetOffsetUtils { private static final Logger LOG = LoggerFactory.getLogger(GetOffsetUtils.class); private static final String HUBSPOT_ID_FIELD = "vid"; + private static final String SALESFORCE_ID_FIELD = "id"; private static final Gson GSON = new Gson(); + public static SerializableFunction getOffsetFnForSalesforce() { + return input -> { + if (input != null) { + try { + HashMap json = + GSON.fromJson(input, new TypeToken>() {}.getType()); + checkArgumentNotNull(json, "Can not get JSON from Salesforce input string"); + Object id = json.get(SALESFORCE_ID_FIELD); + checkArgumentNotNull(id, "Can not get ID from Salesforce input string"); + return ((Double) id).longValue(); + } catch (Exception e) { + LOG.error("Can not get offset from json", e); + } + } + return 0L; + }; + } + public static SerializableFunction getOffsetFnForHubspot() { return input -> { if (input != null) { diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index 1ee536c1ea59..9211f528721b 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -22,6 +22,7 @@ import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; import io.cdap.plugin.hubspot.source.streaming.PullFrequency; import io.cdap.plugin.salesforce.SalesforceConstants; +import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; import io.cdap.plugin.salesforce.plugin.sink.batch.ErrorHandling; import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceSinkConfig; import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; @@ -29,12 +30,7 @@ import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSourceConfig; import io.cdap.plugin.zendesk.source.common.config.BaseZendeskSourceConfig; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; -import org.apache.beam.examples.complete.cdap.options.CdapHubspotStreamingSourceOptions; -import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSinkOptions; -import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSourceOptions; -import org.apache.beam.examples.complete.cdap.options.CdapServiceNowOptions; -import org.apache.beam.examples.complete.cdap.options.CdapZendeskOptions; +import org.apache.beam.examples.complete.cdap.options.*; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; /** @@ -43,6 +39,8 @@ */ public class PluginConfigOptionsConverter { + private static final String SALESFORCE_STREAMING_PUSH_TOPIC_NAME = "pushTopicName"; + public static Map hubspotOptionsToParamsMap(CdapHubspotOptions options) { String apiServerUrl = options.getApiServerUrl(); ImmutableMap.Builder builder = @@ -76,7 +74,15 @@ public static Map serviceNowOptionsToParamsMap(CdapServiceNowOpt .build(); } - public static Map salesforceSourceOptionsToParamsMap( + public static Map salesforceStreamingSourceOptionsToParamsMap( + CdapSalesforceStreamingSourceOptions options) { + return ImmutableMap.builder() + .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) + .put(SALESFORCE_STREAMING_PUSH_TOPIC_NAME, options.getPushTopicName()) + .build(); + } + + public static Map salesforceBatchSourceOptionsToParamsMap( CdapSalesforceSourceOptions options) { return ImmutableMap.builder() .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) @@ -90,7 +96,7 @@ public static Map salesforceSourceOptionsToParamsMap( .build(); } - public static Map salesforceSinkOptionsToParamsMap( + public static Map salesforceBatchSinkOptionsToParamsMap( CdapSalesforceSinkOptions options) { return ImmutableMap.builder() .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) @@ -127,4 +133,14 @@ public static Map zendeskOptionsToParamsMap(CdapZendeskOptions z .put(BaseZendeskSourceConfig.PROPERTY_OBJECTS_TO_PULL, zendeskOptions.getObjectsToPull()) .build(); } + + public static AuthenticatorCredentials getSalesforceStreamingAuthenticatorCredentials( + CdapSalesforceStreamingSourceOptions options) { + return new AuthenticatorCredentials( + options.getUsername(), + options.getPassword(), + options.getConsumerKey(), + options.getConsumerSecret(), + options.getLoginUrl()); + } } From b44af4dc030c035b5969ee94a5012b411df74b65 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Nov 2022 18:26:15 +0400 Subject: [PATCH 14/21] Fix Salesforce Streaming example --- examples/java/cdap/build.gradle | 2 +- .../cdap/CdapSalesforceStreamingToTxt.java | 9 +---- .../cdap/transforms/FormatInputTransform.java | 19 ++++++----- .../complete/cdap/utils/GetOffsetUtils.java | 12 +++++-- .../utils/PluginConfigOptionsConverter.java | 33 +++++++++++-------- .../example-txt-to-cdap-hubspot-companies.txt | 0 .../example-txt-to-cdap-salesforce-custom.txt | 0 7 files changed, 42 insertions(+), 33 deletions(-) rename examples/java/{ => cdap}/src/test/resources/example-txt-to-cdap-hubspot-companies.txt (100%) rename examples/java/{ => cdap}/src/test/resources/example-txt-to-cdap-salesforce-custom.txt (100%) diff --git a/examples/java/cdap/build.gradle b/examples/java/cdap/build.gradle index b070c95a5548..ba4d8f647f1e 100644 --- a/examples/java/cdap/build.gradle +++ b/examples/java/cdap/build.gradle @@ -61,7 +61,7 @@ dependencies { implementation library.java.cdap_plugin_zendesk //TODO: modify to 'implementation library.java.cdap_plugin_salesforce', // when new release with HasOffset interface will be published - implementation "com.akvelon:cdap-salesforce-plugins:1.5.2" + implementation "com.akvelon:cdap-salesforce-plugins:1.4.4" implementation library.java.google_code_gson implementation library.java.hadoop_common implementation library.java.slf4j_api diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java index 2815e6ab92bd..cf5bbdc2f444 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java @@ -17,7 +17,6 @@ */ package org.apache.beam.examples.complete.cdap; -import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; import java.util.Map; import org.apache.beam.examples.complete.cdap.options.CdapSalesforceStreamingSourceOptions; import org.apache.beam.examples.complete.cdap.transforms.FormatInputTransform; @@ -121,9 +120,6 @@ public static PipelineResult run( PluginConfigOptionsConverter.salesforceStreamingSourceOptionsToParamsMap(options); LOG.info("Starting Cdap-Salesforce-streaming-to-txt pipeline with parameters: {}", paramsMap); - AuthenticatorCredentials salesforceCredentials = - PluginConfigOptionsConverter.getSalesforceStreamingAuthenticatorCredentials(options); - /* * Steps: * 1) Read messages in from Cdap Salesforce @@ -135,10 +131,7 @@ public static PipelineResult run( .apply( "readFromCdapSalesforceStreaming", FormatInputTransform.readFromCdapSalesforceStreaming( - paramsMap, - salesforceCredentials, - options.getPullFrequencySec(), - options.getStartOffset())) + paramsMap, options.getPullFrequencySec(), options.getStartOffset())) .setCoder( KvCoder.of( NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java index 2ac529cec381..00ba4c10e7e2 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/transforms/FormatInputTransform.java @@ -27,7 +27,6 @@ import io.cdap.plugin.hubspot.source.streaming.HubspotReceiver; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSource; import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; -import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceBatchSource; import io.cdap.plugin.salesforce.plugin.source.batch.SalesforceSourceConfig; import io.cdap.plugin.salesforce.plugin.source.streaming.SalesforceReceiver; @@ -173,16 +172,12 @@ public static CdapIO.Read readFromCdapSalesforce( * Configures Cdap Salesforce Streaming Read transform. * * @param pluginConfigParams Cdap Hubspot plugin config parameters - * @param salesforceCredentials Authenticator credentials for Salesforce * @param pullFrequencySec Delay in seconds between polling for new records updates * @param startOffset Inclusive start offset from which the reading should be started * @return configured Read transform */ public static CdapIO.Read readFromCdapSalesforceStreaming( - Map pluginConfigParams, - AuthenticatorCredentials salesforceCredentials, - Long pullFrequencySec, - Long startOffset) { + Map pluginConfigParams, Long pullFrequencySec, Long startOffset) { final SalesforceStreamingSourceConfig pluginConfig = new ConfigWrapper<>(SalesforceStreamingSourceConfig.class) @@ -190,6 +185,8 @@ public static CdapIO.Read readFromCdapSalesforceStreaming( .build(); checkStateNotNull(pluginConfig, "Plugin config can't be null."); + pluginConfig.ensurePushTopicExistAndWithCorrectFields(); + CdapIO.Read read = CdapIO.read() .withCdapPlugin( @@ -197,8 +194,14 @@ public static CdapIO.Read readFromCdapSalesforceStreaming( SalesforceStreamingSource.class, GetOffsetUtils.getOffsetFnForSalesforce(), SalesforceReceiver.class, - config -> - new Object[] {salesforceCredentials, pluginConfig.getPushTopicName()})) + config -> { + SalesforceStreamingSourceConfig salesforceConfig = + (SalesforceStreamingSourceConfig) config; + return new Object[] { + salesforceConfig.getAuthenticatorCredentials(), + salesforceConfig.getPushTopicName() + }; + })) .withPluginConfig(pluginConfig) .withKeyClass(NullWritable.class) .withValueClass(String.class); diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java index 12bdbcf1335d..d4f9a9f26cf1 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java @@ -20,6 +20,7 @@ import static org.apache.beam.sdk.util.Preconditions.checkArgumentNotNull; import com.google.gson.Gson; +import com.google.gson.internal.LinkedTreeMap; import java.util.HashMap; import org.apache.beam.sdk.io.cdap.Plugin; import org.apache.beam.sdk.transforms.SerializableFunction; @@ -36,9 +37,13 @@ public class GetOffsetUtils { private static final Logger LOG = LoggerFactory.getLogger(GetOffsetUtils.class); private static final String HUBSPOT_ID_FIELD = "vid"; - private static final String SALESFORCE_ID_FIELD = "id"; + + private static final String SALESFORCE_ID_FIELD = "RecordId__c"; + private static final String SALESFORCE_S_OBJECT = "sobject"; + private static final Gson GSON = new Gson(); + @SuppressWarnings({"rawtypes"}) public static SerializableFunction getOffsetFnForSalesforce() { return input -> { if (input != null) { @@ -46,9 +51,10 @@ public static SerializableFunction getOffsetFnForSalesforce() { HashMap json = GSON.fromJson(input, new TypeToken>() {}.getType()); checkArgumentNotNull(json, "Can not get JSON from Salesforce input string"); - Object id = json.get(SALESFORCE_ID_FIELD); + LinkedTreeMap fieldMap = (LinkedTreeMap) json.get(SALESFORCE_S_OBJECT); + Object id = fieldMap.get(SALESFORCE_ID_FIELD); checkArgumentNotNull(id, "Can not get ID from Salesforce input string"); - return ((Double) id).longValue(); + return Long.parseLong((String) id); } catch (Exception e) { LOG.error("Can not get offset from json", e); } diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index 9211f528721b..38272a07150c 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -22,7 +22,6 @@ import io.cdap.plugin.hubspot.source.streaming.HubspotStreamingSourceConfig; import io.cdap.plugin.hubspot.source.streaming.PullFrequency; import io.cdap.plugin.salesforce.SalesforceConstants; -import io.cdap.plugin.salesforce.authenticator.AuthenticatorCredentials; import io.cdap.plugin.salesforce.plugin.sink.batch.ErrorHandling; import io.cdap.plugin.salesforce.plugin.sink.batch.SalesforceSinkConfig; import io.cdap.plugin.salesforce.plugin.source.batch.util.SalesforceSourceConstants; @@ -39,7 +38,15 @@ */ public class PluginConfigOptionsConverter { + // private static final Logger LOG = LoggerFactory.getLogger(PluginConfigOptionsConverter.class); + private static final String SALESFORCE_STREAMING_PUSH_TOPIC_NAME = "pushTopicName"; + private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_CREATE = "pushTopicNotifyCreate"; + private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_UPDATE = "pushTopicNotifyUpdate"; + private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_DELETE = "pushTopicNotifyDelete"; + private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_FOR_FIELDS = "pushTopicNotifyForFields"; + private static final String SALESFORCE_REFERENCED_NOTIFY_FOR_FIELDS = "Referenced"; + private static final String SALESFORCE_ENABLED_NOTIFY = "Enabled"; public static Map hubspotOptionsToParamsMap(CdapHubspotOptions options) { String apiServerUrl = options.getApiServerUrl(); @@ -53,8 +60,7 @@ public static Map hubspotOptionsToParamsMap(CdapHubspotOptions o .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()); if (options instanceof CdapHubspotStreamingSourceOptions) { // Hubspot PullFrequency value will be ignored as pull frequency is implemented differently in - // CdapIO, - // but it still needs to be passed for the plugin to work correctly. + // CdapIO, but it still needs to be passed for the plugin to work correctly. builder.put(HubspotStreamingSourceConfig.PULL_FREQUENCY, PullFrequency.MINUTES_15.getName()); } return builder.build(); @@ -79,6 +85,17 @@ public static Map salesforceStreamingSourceOptionsToParamsMap( return ImmutableMap.builder() .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()) .put(SALESFORCE_STREAMING_PUSH_TOPIC_NAME, options.getPushTopicName()) + .put(SalesforceConstants.PROPERTY_USERNAME, options.getUsername()) + .put(SalesforceConstants.PROPERTY_PASSWORD, options.getPassword()) + .put(SalesforceConstants.PROPERTY_SECURITY_TOKEN, options.getSecurityToken()) + .put(SalesforceConstants.PROPERTY_CONSUMER_KEY, options.getConsumerKey()) + .put(SalesforceConstants.PROPERTY_CONSUMER_SECRET, options.getConsumerSecret()) + .put(SalesforceConstants.PROPERTY_LOGIN_URL, options.getLoginUrl()) + .put(SalesforceSourceConstants.PROPERTY_SOBJECT_NAME, options.getSObjectName()) + .put(SALESFORCE_PUSH_TOPIC_NOTIFY_CREATE, SALESFORCE_ENABLED_NOTIFY) + .put(SALESFORCE_PUSH_TOPIC_NOTIFY_UPDATE, SALESFORCE_ENABLED_NOTIFY) + .put(SALESFORCE_PUSH_TOPIC_NOTIFY_DELETE, SALESFORCE_ENABLED_NOTIFY) + .put(SALESFORCE_PUSH_TOPIC_NOTIFY_FOR_FIELDS, SALESFORCE_REFERENCED_NOTIFY_FOR_FIELDS) .build(); } @@ -133,14 +150,4 @@ public static Map zendeskOptionsToParamsMap(CdapZendeskOptions z .put(BaseZendeskSourceConfig.PROPERTY_OBJECTS_TO_PULL, zendeskOptions.getObjectsToPull()) .build(); } - - public static AuthenticatorCredentials getSalesforceStreamingAuthenticatorCredentials( - CdapSalesforceStreamingSourceOptions options) { - return new AuthenticatorCredentials( - options.getUsername(), - options.getPassword(), - options.getConsumerKey(), - options.getConsumerSecret(), - options.getLoginUrl()); - } } diff --git a/examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt b/examples/java/cdap/src/test/resources/example-txt-to-cdap-hubspot-companies.txt similarity index 100% rename from examples/java/src/test/resources/example-txt-to-cdap-hubspot-companies.txt rename to examples/java/cdap/src/test/resources/example-txt-to-cdap-hubspot-companies.txt diff --git a/examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt b/examples/java/cdap/src/test/resources/example-txt-to-cdap-salesforce-custom.txt similarity index 100% rename from examples/java/src/test/resources/example-txt-to-cdap-salesforce-custom.txt rename to examples/java/cdap/src/test/resources/example-txt-to-cdap-salesforce-custom.txt From 6e269309a44cd619a38ec3b1e17cbcc399fe9e3b Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Nov 2022 19:38:57 +0400 Subject: [PATCH 15/21] Add javadoc, readme --- .../beam/examples/complete/cdap/README.md | 40 +++++++++++++++++++ .../complete/cdap/utils/GetOffsetUtils.java | 11 +++-- .../utils/PluginConfigOptionsConverter.java | 16 ++++++-- .../cdap/utils/StructuredRecordUtils.java | 1 + 4 files changed, 62 insertions(+), 6 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 6703e54c65ac..1bd89af8d587 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -200,6 +200,46 @@ To execute this pipeline, specify the parameters in the following format: ``` Please see CDAP [Salesforce Batch Sink](https://github.com/data-integrations/salesforce/blob/develop/docs/Salesforce-batchsink.md) for more information. +## Running the CdapSalesforceStreamingToTxt pipeline example + +Gradle 'executeCdap' task allows to run the pipeline via the following command: + +```bash +gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.CdapSalesforceStreamingToTxt \ + -Dexec.args="--= --=" +``` + +`CdapSalesforceStreamingToTxt` pipeline parameters: +- `username` - Salesforce username. +- `password` - Salesforce user password. +- `securityToken` - Salesforce security token. +- `consumerKey` - Salesforce connected app's consumer key. +- `consumerSecret` - Salesforce connected app's consumer secret. +- `loginUrl` - Salesforce endpoint to authenticate to. Example: *'https://MyDomainName.my.salesforce.com/services/oauth2/token'*. +- `sObjectName` - Salesforce object to pull supported by CDAP Salesforce Streaming Source. +- `pushTopicName` - name of the push topic that was created from query for some sObject. +If push topic with such name doesn't exist, then new push topic for provided **'sObjectName'** will be created. +- `pullFrequencySec` - delay in seconds between polling for new records updates. (Optional) +- `startOffset` - inclusive start offset from which the reading should be started. (Optional) + +Please see [CDAP Salesforce](https://github.com/data-integrations/salesforce) for more information. +Also, please see documentation regarding Salesforce streaming API authorization [here](https://developer.salesforce.com/docs/atlas.en-us.api_streaming.meta/api_streaming/code_sample_auth_oauth.htm). + +To execute this pipeline, specify the parameters in the following format: + +```bash + --username=your-user-name\ + --password=your-password \ + --securityToken=your-token \ + --consumerKey=your-key \ + --consumerSecret=your-secret \ + --loginUrl=your-login-url \ + --sObjectName=object-name \ + --pushTopicName=your-topic-name \ + --pullFrequencySec=100 \ + --startOffset=1000 +``` + ## Running the CdapZendeskToTxt pipeline example Gradle 'executeCdap' task allows to run the pipeline via the following command: diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java index d4f9a9f26cf1..3aee42c70f07 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java @@ -35,14 +35,16 @@ public class GetOffsetUtils { private static final Logger LOG = LoggerFactory.getLogger(GetOffsetUtils.class); + private static final Gson GSON = new Gson(); private static final String HUBSPOT_ID_FIELD = "vid"; - private static final String SALESFORCE_ID_FIELD = "RecordId__c"; private static final String SALESFORCE_S_OBJECT = "sobject"; - private static final Gson GSON = new Gson(); - + /** + * Function for getting offset for Salesforce record that has custom number {@link + * #SALESFORCE_ID_FIELD} field. + */ @SuppressWarnings({"rawtypes"}) public static SerializableFunction getOffsetFnForSalesforce() { return input -> { @@ -63,6 +65,9 @@ public static SerializableFunction getOffsetFnForSalesforce() { }; } + /** + * Function for getting offset for Hubspot record that has {@link #HUBSPOT_ID_FIELD} number field. + */ public static SerializableFunction getOffsetFnForHubspot() { return input -> { if (input != null) { diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index 38272a07150c..1dd1acd895d8 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -29,7 +29,13 @@ import io.cdap.plugin.zendesk.source.batch.ZendeskBatchSourceConfig; import io.cdap.plugin.zendesk.source.common.config.BaseZendeskSourceConfig; import java.util.Map; -import org.apache.beam.examples.complete.cdap.options.*; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotOptions; +import org.apache.beam.examples.complete.cdap.options.CdapHubspotStreamingSourceOptions; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSinkOptions; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceSourceOptions; +import org.apache.beam.examples.complete.cdap.options.CdapSalesforceStreamingSourceOptions; +import org.apache.beam.examples.complete.cdap.options.CdapServiceNowOptions; +import org.apache.beam.examples.complete.cdap.options.CdapZendeskOptions; import org.apache.beam.vendor.guava.v26_0_jre.com.google.common.collect.ImmutableMap; /** @@ -38,8 +44,6 @@ */ public class PluginConfigOptionsConverter { - // private static final Logger LOG = LoggerFactory.getLogger(PluginConfigOptionsConverter.class); - private static final String SALESFORCE_STREAMING_PUSH_TOPIC_NAME = "pushTopicName"; private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_CREATE = "pushTopicNotifyCreate"; private static final String SALESFORCE_PUSH_TOPIC_NOTIFY_UPDATE = "pushTopicNotifyUpdate"; @@ -48,6 +52,7 @@ public class PluginConfigOptionsConverter { private static final String SALESFORCE_REFERENCED_NOTIFY_FOR_FIELDS = "Referenced"; private static final String SALESFORCE_ENABLED_NOTIFY = "Enabled"; + /** Returns map of parameters for Cdap Hubspot plugins. */ public static Map hubspotOptionsToParamsMap(CdapHubspotOptions options) { String apiServerUrl = options.getApiServerUrl(); ImmutableMap.Builder builder = @@ -66,6 +71,7 @@ public static Map hubspotOptionsToParamsMap(CdapHubspotOptions o return builder.build(); } + /** Returns map of parameters for Cdap ServiceNow plugin. */ public static Map serviceNowOptionsToParamsMap(CdapServiceNowOptions options) { return ImmutableMap.builder() .put(ServiceNowConstants.PROPERTY_CLIENT_ID, options.getClientId()) @@ -80,6 +86,7 @@ public static Map serviceNowOptionsToParamsMap(CdapServiceNowOpt .build(); } + /** Returns map of parameters for Cdap Salesforce streaming source plugin. */ public static Map salesforceStreamingSourceOptionsToParamsMap( CdapSalesforceStreamingSourceOptions options) { return ImmutableMap.builder() @@ -99,6 +106,7 @@ public static Map salesforceStreamingSourceOptionsToParamsMap( .build(); } + /** Returns map of parameters for Cdap Salesforce batch source plugin. */ public static Map salesforceBatchSourceOptionsToParamsMap( CdapSalesforceSourceOptions options) { return ImmutableMap.builder() @@ -113,6 +121,7 @@ public static Map salesforceBatchSourceOptionsToParamsMap( .build(); } + /** Returns map of parameters for Cdap Salesforce batch sink plugin. */ public static Map salesforceBatchSinkOptionsToParamsMap( CdapSalesforceSinkOptions options) { return ImmutableMap.builder() @@ -133,6 +142,7 @@ public static Map salesforceBatchSinkOptionsToParamsMap( .build(); } + /** Returns map of parameters for Cdap Zendesk plugin. */ public static Map zendeskOptionsToParamsMap(CdapZendeskOptions zendeskOptions) { return ImmutableMap.builder() .put(Constants.Reference.REFERENCE_NAME, zendeskOptions.getReferenceName()) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java index 6cb8df6a8a95..5d1ae297b5f7 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/StructuredRecordUtils.java @@ -27,6 +27,7 @@ */ public class StructuredRecordUtils { + /** Converts {@link StructuredRecord} to String json-like format. */ public static String structuredRecordToString(StructuredRecord structuredRecord) { if (structuredRecord == null) { return "{}"; From 2152094ad19c6036ad0c9c4361db366028607c54 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Nov 2022 19:44:52 +0400 Subject: [PATCH 16/21] Fix readme for Salesforce streaming example --- .../java/org/apache/beam/examples/complete/cdap/README.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 1bd89af8d587..22d973917731 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -217,8 +217,9 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap - `consumerSecret` - Salesforce connected app's consumer secret. - `loginUrl` - Salesforce endpoint to authenticate to. Example: *'https://MyDomainName.my.salesforce.com/services/oauth2/token'*. - `sObjectName` - Salesforce object to pull supported by CDAP Salesforce Streaming Source. -- `pushTopicName` - name of the push topic that was created from query for some sObject. -If push topic with such name doesn't exist, then new push topic for provided **'sObjectName'** will be created. +**Important note:** provided *sObject* should have *'RecordId__c'* custom auto-generated number field in order to correctly run the example. +- `pushTopicName` - name of the push topic that was created from query for some sObject. This push topic should have enabled *pushTopicNotifyCreate* property. +If push topic with such name doesn't exist, then new push topic for provided **'sObjectName'** will be created automatically. - `pullFrequencySec` - delay in seconds between polling for new records updates. (Optional) - `startOffset` - inclusive start offset from which the reading should be started. (Optional) From 1e4b1a704da9e532d308e11192014910084cd140 Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Nov 2022 19:48:47 +0400 Subject: [PATCH 17/21] Fix javadoc for Salesforce Streaming example --- .../examples/complete/cdap/CdapSalesforceStreamingToTxt.java | 1 + 1 file changed, 1 insertion(+) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java index cf5bbdc2f444..d1cbf7a645ef 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapSalesforceStreamingToTxt.java @@ -76,6 +76,7 @@ * --consumerSecret=your-secret \ * --loginUrl=your-login-url \ * --sObjectName=object-name \ + * --pushTopicName=your-push-topic-name \ * --referenceName=your-reference-name \ * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \ * --pullFrequencySec=1 \ From 6fe83ddbfcfec14245bda80439a97ac7dd44561f Mon Sep 17 00:00:00 2001 From: "vitaly.terentyev" Date: Wed, 16 Nov 2022 20:09:41 +0400 Subject: [PATCH 18/21] Fix Spotless --- .../beam/examples/complete/cdap/utils/GetOffsetUtils.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java index 3aee42c70f07..176bb3d7675d 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/GetOffsetUtils.java @@ -54,9 +54,11 @@ public static SerializableFunction getOffsetFnForSalesforce() { GSON.fromJson(input, new TypeToken>() {}.getType()); checkArgumentNotNull(json, "Can not get JSON from Salesforce input string"); LinkedTreeMap fieldMap = (LinkedTreeMap) json.get(SALESFORCE_S_OBJECT); - Object id = fieldMap.get(SALESFORCE_ID_FIELD); - checkArgumentNotNull(id, "Can not get ID from Salesforce input string"); - return Long.parseLong((String) id); + if (fieldMap != null) { + Object id = fieldMap.get(SALESFORCE_ID_FIELD); + checkArgumentNotNull(id, "Can not get ID from Salesforce input string"); + return Long.parseLong((String) id); + } } catch (Exception e) { LOG.error("Can not get offset from json", e); } From 61cf3d53771b8c117943fe2bcb32766bdc096a90 Mon Sep 17 00:00:00 2001 From: Elizaveta Lomteva Date: Wed, 23 Nov 2022 19:44:53 +0400 Subject: [PATCH 19/21] replace api key with auth token --- examples/java/cdap/build.gradle | 5 +++-- .../complete/cdap/CdapHubspotStreamingToTxt.java | 2 +- .../examples/complete/cdap/CdapHubspotToTxt.java | 2 +- .../apache/beam/examples/complete/cdap/README.md | 16 ++++++++-------- .../examples/complete/cdap/TxtToCdapHubspot.java | 2 +- .../cdap/options/CdapHubspotOptions.java | 6 +++--- .../cdap/utils/PluginConfigOptionsConverter.java | 2 +- 7 files changed, 18 insertions(+), 17 deletions(-) diff --git a/examples/java/cdap/build.gradle b/examples/java/cdap/build.gradle index ba4d8f647f1e..dc1c16c8e8e7 100644 --- a/examples/java/cdap/build.gradle +++ b/examples/java/cdap/build.gradle @@ -56,7 +56,7 @@ dependencies { implementation library.java.cdap_hydrator_common //TODO: modify to 'implementation library.java.cdap_plugin_hubspot', // when new release with HasOffset interface will be published - implementation "com.akvelon:cdap-hubspot-plugins:1.0.3" + implementation "com.akvelon:cdap-hubspot-plugins:1.1.0" implementation library.java.cdap_plugin_service_now implementation library.java.cdap_plugin_zendesk //TODO: modify to 'implementation library.java.cdap_plugin_salesforce', @@ -66,6 +66,7 @@ dependencies { implementation library.java.hadoop_common implementation library.java.slf4j_api implementation library.java.vendored_guava_26_0_jre + implementation project(path: ":runners:google-cloud-dataflow-java") runtimeOnly project(path: ":runners:direct-java", configuration: "shadow") testImplementation project(path: ":runners:direct-java", configuration: "shadow") testImplementation library.java.hamcrest @@ -87,7 +88,7 @@ dependencies { */ def preCommitRunnerClass = [ directRunner: "org.apache.beam.runners.direct.DirectRunner", - flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner", + flinkRunner: "org.apache.beam.runners.flink.TestFlinkRunner" ] for (String runner : preCommitRunners) { diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java index e39cfaaea072..5f75326fa514 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotStreamingToTxt.java @@ -69,7 +69,7 @@ * # Running the pipeline * To execute this pipeline, specify the parameters in the following format: * {@code - * --apikey=your-api-key \ + * --authToken=your-private-app-access-token \ * --referenceName=your-reference-name \ * --objectType=Contacts \ * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \ diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java index edc1779b0187..0a47184b9d28 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/CdapHubspotToTxt.java @@ -68,7 +68,7 @@ * # Running the pipeline * To execute this pipeline, specify the parameters in the following format: * {@code - * --apikey=your-api-key \ + * --authToken=your-private-app-access-token \ * --referenceName=your-reference-name \ * --objectType=Contacts \ * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index 22d973917731..ef29e85ef757 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -49,7 +49,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap ``` `CdapHubspotToTxt` pipeline parameters: -- `apikey` - Hubspot OAuth2 API Key +- `authToken` - Hubspot Private Application Access token - `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) - `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. @@ -58,7 +58,7 @@ Please see CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubs To execute this pipeline, specify the parameters in the following format: ```bash - --apikey=your-api-key \ + --authToken=your-private-app-access-token \ --referenceName=your-reference-name \ --objectType=Contacts \ --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix @@ -74,7 +74,7 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap ``` `CdapHubspotStreamingToTxt` pipeline parameters: -- `apikey` - Hubspot OAuth2 API Key +- `authToken` - Hubspot Private Application Access token - `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) - `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. - `pullFrequencySec` - delay in seconds between polling for new records updates. (Optional) @@ -85,7 +85,7 @@ Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/ To execute this pipeline, specify the parameters in the following format: ```bash - --apikey=your-api-key \ + --authToken=your-private-app-access-token \ --referenceName=your-reference-name \ --objectType=Contacts \ --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix \ @@ -103,17 +103,17 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtT ``` `TxtToCdapHubspot` pipeline parameters: -- `apikey` - Hubspot OAuth2 API Key -- `objectType` - Hubspot objects to pull supported by [Hubspot Streaming Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) +- `authToken` - Hubspot Private Application Access token +- `objectType` - Hubspot objects to pull supported by [Hubspot Batch Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) - `inputTxtFilePath` - input .txt file path - `locksDirPath` - locks directory path where locks will be stored. This parameter is needed for Hadoop External Synchronization (mechanism for acquiring locks related to the write job). -Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. +Please see CDAP [Hubspot Batch Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) for more information. To execute this pipeline, specify the parameters in the following format: ```bash - --apikey=your-api-key \ + --authToken=your-private-app-access-token \ --referenceName=your-reference-name \ --objectType=your-object-type \ --inputTxtFilePath=your-path-to-input-txt-file \ diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java index 6d7edd9c151d..e7a9b4f084ea 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/TxtToCdapHubspot.java @@ -64,7 +64,7 @@ * # Running the pipeline * To execute this pipeline, specify the parameters in the following format: * {@code - * --apikey=your-api-key \ + * --authToken=your-private-app-access-token \ * --referenceName=your-reference-name \ * --objectType=your-object-type \ * --inputTxtFilePath=your-path-to-input-file \ diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java index bb49e0311ed7..9a210b1c2b39 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/options/CdapHubspotOptions.java @@ -32,10 +32,10 @@ public interface CdapHubspotOptions extends BaseCdapOptions { void setApiServerUrl(String apiServerUrl); @Validation.Required - @Description("Hubspot OAuth2 API Key.") - String getApiKey(); + @Description("Hubspot Private Application Access Token.") + String getAuthToken(); - void setApiKey(String apiKey); + void setAuthToken(String authToken); @Validation.Required @Description("Name of object to pull from Hubspot (e.g. Contacts).") diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java index 1dd1acd895d8..00e79a42fbd2 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/utils/PluginConfigOptionsConverter.java @@ -60,7 +60,7 @@ public static Map hubspotOptionsToParamsMap(CdapHubspotOptions o .put( BaseHubspotConfig.API_SERVER_URL, apiServerUrl != null ? apiServerUrl : BaseHubspotConfig.DEFAULT_API_SERVER_URL) - .put(BaseHubspotConfig.API_KEY, options.getApiKey()) + .put(BaseHubspotConfig.AUTHORIZATION_TOKEN, options.getAuthToken()) .put(BaseHubspotConfig.OBJECT_TYPE, options.getObjectType()) .put(Constants.Reference.REFERENCE_NAME, options.getReferenceName()); if (options instanceof CdapHubspotStreamingSourceOptions) { From 5996d8005dd80f85a22a1e0fc7129c3bbe49e6d0 Mon Sep 17 00:00:00 2001 From: Elizaveta Lomteva Date: Wed, 23 Nov 2022 19:48:32 +0400 Subject: [PATCH 20/21] updated README.md --- .../org/apache/beam/examples/complete/cdap/README.md | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md index ef29e85ef757..b612fa93d02c 100644 --- a/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md +++ b/examples/java/cdap/src/main/java/org/apache/beam/examples/complete/cdap/README.md @@ -50,10 +50,10 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap `CdapHubspotToTxt` pipeline parameters: - `authToken` - Hubspot Private Application Access token -- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) +- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Batch Source](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-batchsource.md) - `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. -Please see CDAP [Hubspot Batch Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsource.md) for more information. +Please see CDAP [Hubspot Batch Source](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-batchsource.md) for more information. To execute this pipeline, specify the parameters in the following format: @@ -75,12 +75,12 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.Cdap `CdapHubspotStreamingToTxt` pipeline parameters: - `authToken` - Hubspot Private Application Access token -- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) +- `objectType` - Hubspot objects to pull supported by CDAP [Hubspot Streaming Source](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-streamingsource.md) - `outputTxtFilePathPrefix` - path to output folder with filename prefix. It will write a set of .txt files with names like {prefix}-###. - `pullFrequencySec` - delay in seconds between polling for new records updates. (Optional) - `startOffset` - inclusive start offset from which the reading should be started. (Optional) -Please see CDAP [Hubspot Streaming Source](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-streamingsource.md) for more information. +Please see CDAP [Hubspot Streaming Source](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-streamingsource.md) for more information. To execute this pipeline, specify the parameters in the following format: @@ -104,11 +104,11 @@ gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.TxtT `TxtToCdapHubspot` pipeline parameters: - `authToken` - Hubspot Private Application Access token -- `objectType` - Hubspot objects to pull supported by [Hubspot Batch Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) +- `objectType` - Hubspot objects to pull supported by [Hubspot Batch Sink](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-batchsink.md) - `inputTxtFilePath` - input .txt file path - `locksDirPath` - locks directory path where locks will be stored. This parameter is needed for Hadoop External Synchronization (mechanism for acquiring locks related to the write job). -Please see CDAP [Hubspot Batch Sink](https://github.com/data-integrations/hubspot/blob/develop/docs/Hubspot-batchsink.md) for more information. +Please see CDAP [Hubspot Batch Sink](https://github.com/akvelon/cdap-hubspot/blob/release-1.1.0-authorization/docs/Hubspot-batchsink.md) for more information. To execute this pipeline, specify the parameters in the following format: From a4ebc842c773f3a7f047c494a5f2959a6114f156 Mon Sep 17 00:00:00 2001 From: Elizaveta Lomteva Date: Tue, 29 Nov 2022 15:06:54 +0400 Subject: [PATCH 21/21] deleted unused dependency --- examples/java/cdap/build.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/examples/java/cdap/build.gradle b/examples/java/cdap/build.gradle index dc1c16c8e8e7..97ee6b73cbba 100644 --- a/examples/java/cdap/build.gradle +++ b/examples/java/cdap/build.gradle @@ -66,7 +66,6 @@ dependencies { implementation library.java.hadoop_common implementation library.java.slf4j_api implementation library.java.vendored_guava_26_0_jre - implementation project(path: ":runners:google-cloud-dataflow-java") runtimeOnly project(path: ":runners:direct-java", configuration: "shadow") testImplementation project(path: ":runners:direct-java", configuration: "shadow") testImplementation library.java.hamcrest