|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | +package org.apache.beam.examples.complete.cdap.zendesk; |
| 19 | + |
| 20 | +import static org.apache.beam.examples.complete.cdap.zendesk.transforms.FormatInputTransform.readFromCdapZendesk; |
| 21 | + |
| 22 | +import io.cdap.cdap.api.data.format.StructuredRecord; |
| 23 | +import java.util.Map; |
| 24 | +import org.apache.beam.examples.complete.cdap.utils.StructuredRecordUtils; |
| 25 | +import org.apache.beam.examples.complete.cdap.zendesk.options.CdapZendeskOptions; |
| 26 | +import org.apache.beam.examples.complete.cdap.zendesk.utils.PluginConfigOptionsConverter; |
| 27 | +import org.apache.beam.sdk.Pipeline; |
| 28 | +import org.apache.beam.sdk.PipelineResult; |
| 29 | +import org.apache.beam.sdk.coders.KvCoder; |
| 30 | +import org.apache.beam.sdk.coders.NullableCoder; |
| 31 | +import org.apache.beam.sdk.coders.SerializableCoder; |
| 32 | +import org.apache.beam.sdk.coders.StringUtf8Coder; |
| 33 | +import org.apache.beam.sdk.io.TextIO; |
| 34 | +import org.apache.beam.sdk.io.hadoop.WritableCoder; |
| 35 | +import org.apache.beam.sdk.options.PipelineOptionsFactory; |
| 36 | +import org.apache.beam.sdk.transforms.MapValues; |
| 37 | +import org.apache.beam.sdk.transforms.Values; |
| 38 | +import org.apache.beam.sdk.values.TypeDescriptors; |
| 39 | +import org.apache.hadoop.io.NullWritable; |
| 40 | +import org.slf4j.Logger; |
| 41 | +import org.slf4j.LoggerFactory; |
| 42 | + |
| 43 | +/** |
| 44 | + * The {@link CdapZendeskToTxt} pipeline is a batch pipeline which ingests data in JSON format from |
| 45 | + * CDAP Zendesk, and outputs the resulting records to .txt file. Zendesk parameters and output txt |
| 46 | + * file path are specified by the user as template parameters. <br> |
| 47 | + * |
| 48 | + * <p><b>Example Usage</b> |
| 49 | + * |
| 50 | + * <pre> |
| 51 | + * # Gradle preparation |
| 52 | + * |
| 53 | + * To run this example your {@code build.gradle} file should contain the following task |
| 54 | + * to execute the pipeline: |
| 55 | + * {@code |
| 56 | + * task executeCdap (type:JavaExec) { |
| 57 | + * mainClass = System.getProperty("mainClass") |
| 58 | + * classpath = sourceSets.main.runtimeClasspath |
| 59 | + * systemProperties System.getProperties() |
| 60 | + * args System.getProperty("exec.args", "").split() |
| 61 | + * } |
| 62 | + * } |
| 63 | + * |
| 64 | + * This task allows to run the pipeline via the following command: |
| 65 | + * {@code |
| 66 | + * gradle clean executeCdap -DmainClass=org.apache.beam.examples.complete.cdap.zendesk.CdapZendeskToTxt \ |
| 67 | + * -Dexec.args="--<argument>=<value> --<argument>=<value>" |
| 68 | + * } |
| 69 | + * |
| 70 | + * # Running the pipeline |
| 71 | + * To execute this pipeline, specify the parameters in the following format: |
| 72 | + * {@code |
| 73 | + * --zendeskBaseUrl=your-url \ |
| 74 | + * --adminEmail=your-email \ |
| 75 | + * --apiToken=your-token \ |
| 76 | + * --objectsToPull=Groups \ |
| 77 | + * --referenceName=your-reference-name \ |
| 78 | + * --outputTxtFilePathPrefix=your-path-to-output-folder-with-filename-prefix |
| 79 | + * } |
| 80 | + * |
| 81 | + * By default this will run the pipeline locally with the DirectRunner. To change the runner, specify: |
| 82 | + * {@code |
| 83 | + * --runner=YOUR_SELECTED_RUNNER |
| 84 | + * } |
| 85 | + * </pre> |
| 86 | + */ |
| 87 | +public class CdapZendeskToTxt { |
| 88 | + |
| 89 | + /* Logger for class.*/ |
| 90 | + private static final Logger LOG = LoggerFactory.getLogger(CdapZendeskToTxt.class); |
| 91 | + |
| 92 | + /** |
| 93 | + * Main entry point for pipeline execution. |
| 94 | + * |
| 95 | + * @param args Command line arguments to the pipeline. |
| 96 | + */ |
| 97 | + public static void main(String[] args) { |
| 98 | + CdapZendeskOptions options = |
| 99 | + PipelineOptionsFactory.fromArgs(args).withValidation().as(CdapZendeskOptions.class); |
| 100 | + |
| 101 | + // Create the pipeline |
| 102 | + Pipeline pipeline = Pipeline.create(options); |
| 103 | + run(pipeline, options); |
| 104 | + } |
| 105 | + |
| 106 | + /** |
| 107 | + * Runs a pipeline which reads message from CDAP and writes it to .txt file. |
| 108 | + * |
| 109 | + * @param options arguments to the pipeline |
| 110 | + */ |
| 111 | + public static PipelineResult run(Pipeline pipeline, CdapZendeskOptions options) { |
| 112 | + Map<String, Object> pluginConfigParams = |
| 113 | + PluginConfigOptionsConverter.zendeskOptionsToParamsMap(options); |
| 114 | + LOG.info("Starting Cdap-Zendesk-To-Txt pipeline with parameters: {}", pluginConfigParams); |
| 115 | + |
| 116 | + /* |
| 117 | + * Steps: |
| 118 | + * 1) Read messages in from Cdap Zendesk |
| 119 | + * 2) Extract values only |
| 120 | + * 3) Write successful records to .txt file |
| 121 | + */ |
| 122 | + |
| 123 | + pipeline |
| 124 | + .apply("readFromCdapZendesk", readFromCdapZendesk(pluginConfigParams)) |
| 125 | + .setCoder( |
| 126 | + KvCoder.of( |
| 127 | + NullableCoder.of(WritableCoder.of(NullWritable.class)), |
| 128 | + SerializableCoder.of(StructuredRecord.class))) |
| 129 | + .apply( |
| 130 | + MapValues.into(TypeDescriptors.strings()) |
| 131 | + .via(StructuredRecordUtils::structuredRecordToString)) |
| 132 | + .setCoder( |
| 133 | + KvCoder.of( |
| 134 | + NullableCoder.of(WritableCoder.of(NullWritable.class)), StringUtf8Coder.of())) |
| 135 | + .apply(Values.create()) |
| 136 | + .apply("writeToTxt", TextIO.write().to(options.getOutputTxtFilePathPrefix())); |
| 137 | + |
| 138 | + return pipeline.run(); |
| 139 | + } |
| 140 | +} |
0 commit comments